├── README ├── calvin.conf ├── data └── .gitignore ├── install-ext └── src ├── Makefile ├── Makefile.default ├── Makefile.template ├── applications ├── Makefile.inc ├── application.h ├── microbenchmark.cc ├── microbenchmark.h ├── tpcc.cc └── tpcc.h ├── backend ├── Makefile.inc ├── simple_storage.cc ├── simple_storage.h ├── storage.h ├── storage_manager.cc └── storage_manager.h ├── common ├── Makefile.inc ├── mutex.h ├── types.h ├── utils.cc └── utils.h ├── log ├── Makefile.inc ├── local_mem_log.cc ├── local_mem_log.h ├── local_paxos.cc ├── local_paxos.h ├── log.h ├── paxos.cc └── paxos.h ├── machine ├── Makefile.inc ├── client.h ├── cluster_config.cc ├── cluster_config.h ├── cluster_manager.cc ├── cluster_manager.h ├── connection.cc ├── connection.h ├── lowlatency_sequencer.cc ├── lowlatency_sequencer.h ├── sequencer.cc ├── sequencer.h └── zmq.hpp ├── proto ├── Makefile.inc ├── cluster_config.proto ├── message.proto ├── scalar.proto └── txn.proto ├── scheduler ├── Makefile.inc ├── deterministic_lock_manager.cc ├── deterministic_lock_manager.h ├── deterministic_scheduler.cc ├── deterministic_scheduler.h ├── lock_manager.h └── scheduler.h └── scripts ├── Makefile.inc ├── calvindb_server.cc ├── cluster.cc ├── lowlatency_calvindb_server.cc ├── script_utils.cc └── script_utils.h /README: -------------------------------------------------------------------------------- 1 | ------------This is the Geo-replicated CalvinDB codebase (under development)------------ 2 | 3 | CalvinDB is a scalable transactional database system that leverages determinism to guarantee active replication and full ACID-compliance of distributed transactions without two-phase commit. 4 | 5 | This codebase supports original CavlinDB and low-latency version of CalvinDB. 6 | 7 | Installation(This is for Ubuntu 14.04, it should be similar with other Linux distribution): 8 | 1. You should install the following softwares: 9 | Git 10 | make,g++ 11 | autoconf, libtool,libreadline-dev,libsnappy-dev, pkg-config 12 | subversion 13 | unzip, tar 14 | 15 | 16 | 2. You should install the following external libraries: 17 | --- protobuf 18 | --- glog 19 | --- zeromq 20 | --- gflags 21 | 22 | I wrote a script(install-ext), so that you can directly run this script to download and compile the libraries. 23 | 24 | 25 | 3. After that, you should add the LD_LIBRARY_PATH to the ~/.bashrc, below is my cofig: 26 | export LD_LIBRARY_PATH=~/CalvinDB/ext/zeromq/src/.libs:~/CalvinDB/ext/protobuf/src/.libs:~/CalvinDB/ext/glog/.libs:~/CalvinDB/ext/gflags/.libs 27 | 28 | Then run source ~/.bashrc. 29 | 30 | 31 | 4. Now you can compile the code: 32 | 33 | $ cd ~/CalvinDB/src 34 | $ make -j 35 | 36 | Then all the binaries will be created at ~/CalvinFS/bin/ 37 | 38 | 5. You can change the config file(calvin.conf) to include all machines in the cluster. 39 | One example config file is below: 40 | 0:0:192.168.122.1:10001 41 | 1:0:192.168.122.2:10001 42 | 2:1:192.168.122.3:10001 43 | 3:1:192.168.122.4:10001 44 | 4:2:192.168.122.5:10001 45 | 5:2:192.168.122.6:10001 46 | 47 | First column: global machine ID; 48 | Second column: replica ID; 49 | Third column: IP address; 50 | Fourth column: port 51 | 52 | This means that we have 3 replicas, and each replica has 2 machines. 53 | 54 | 6. Now you can run CalvinDB: 55 | 56 | $ cd ~/CalvinDB 57 | $ bin/scripts/cluster --command="status" --type=1 // Check the status of the cluster if 6 replicas (type=0 if 3 replicas) 58 | $ bin/scripts/cluster --command="update" --type=1 // Use git to get the latest code, and compile it on each machine 59 | $ bin/scripts/cluster --command="put-config" --type=1// Send the config file to all the machines in the cluster 60 | 61 | // Run Original CalvinDB with 3 replicas 62 | $ bin/scripts/cluster --command="start" --lowlatency=0 --type=0 --experiment=0 --percent_mp=0 --percent_mr=0 --hot_records=10000 --max_batch_size=100 63 | // Run Original CalvinDB with 6 replicas 64 | $ bin/scripts/cluster --command="start" --lowlatency=0 --type=1 --experiment=0 --percent_mp=0 --percent_mr=0 --hot_records=10000 --max_batch_size=100 65 | 66 | // Run Lowlatency CalvinDB with 3 replicas 67 | $ bin/scripts/cluster --command="start" --lowlatency=1 --type=0 --experiment=0 --percent_mp=0 --percent_mr=0 --hot_records=10000 --max_batch_size=100 68 | // Run Lowlatency CalvinDB with 6 replicas 69 | $ bin/scripts/cluster --command="start" --lowlatency=1 --type=1 --experiment=0 --percent_mp=0 --percent_mr=0 --hot_records=10000 --max_batch_size=100 70 | // Run Lowlatency CalvinDB with 6 replicas (strong availbility) 71 | $ bin/scripts/cluster --command="start" --lowlatency=1 --type=2 --experiment=0 --percent_mp=0 --percent_mr=0 --hot_records=10000 --max_batch_size=100 72 | 73 | You probably need another terminal to terminate the db process when complete your test by running: 74 | $ bin/scripts/cluster --command="kill" --lowlatency=0 --type=1// lowlatency=0: CalvinDB; lowlatency=1 or 2: Low latency CavlinDB; type=1: 6 replicas; type=0: 3 replicas 75 | 76 | Will be continued. 77 | 78 | 79 | 80 | -------------------------------------------------------------------------------- /calvin.conf: -------------------------------------------------------------------------------- 1 | 0:0:52.207.255.170:10001 2 | 1:0:34.239.130.135:10001 3 | 2:0:34.227.105.177:10001 4 | 3:0:34.230.73.19:10001 5 | 4:1:34.217.121.245:10001 6 | 5:1:54.187.144.183:10001 7 | 6:1:54.213.116.105:10001 8 | 7:1:52.26.158.141:10001 9 | 8:2:54.194.67.47:10001 10 | 9:2:54.171.87.178:10001 11 | 10:2:34.240.203.152:10001 12 | 11:2:34.244.64.246:10001 13 | 12:3:18.188.143.35:10001 14 | 13:3:18.219.62.211:10001 15 | 14:3:13.58.173.250:10001 16 | 15:3:13.59.115.136:10001 17 | 16:4:54.153.108.87:10001 18 | 17:4:13.57.222.173:10001 19 | 18:4:54.241.136.139:10001 20 | 19:4:13.57.181.49:10001 21 | 20:5:35.178.93.17:10001 22 | 21:5:35.177.187.32:10001 23 | 22:5:35.177.56.151:10001 24 | 23:5:35.177.68.103:10001 25 | 26 | 27 | 28 | 29 | 30 | 31 | 32 | 33 | 34 | 35 | 36 | 37 | 38 | 39 | 40 | -------------------------------------------------------------------------------- /data/.gitignore: -------------------------------------------------------------------------------- 1 | # Ignore everything in this directory 2 | * 3 | # Except this file 4 | !.gitignore 5 | -------------------------------------------------------------------------------- /install-ext: -------------------------------------------------------------------------------- 1 | #!/bin/bash 2 | 3 | # requires: make, g++, autoconf, libtool, libreadline-dev, svn(subversion),git,unzip,tar,cmake, libsnappy-dev 4 | mkdir ext 5 | cd ext 6 | 7 | # protobuf 8 | wget https://github.com/google/protobuf/releases/download/v3.2.0/protobuf-cpp-3.2.0.tar.gz 9 | tar -xzf protobuf-cpp-3.2.0.tar.gz 10 | mv protobuf-3.2.0/ protobuf 11 | rm -fr protobuf-cpp-3.2.0.tar.gz 12 | cd protobuf 13 | ./autogen.sh 14 | ./configure --prefix=$PWD 15 | make -j 16 | make install 17 | cd .. 18 | 19 | 20 | # glog 21 | wget https://github.com/google/glog/archive/v0.3.4.tar.gz 22 | tar -xzf v0.3.4.tar.gz 23 | mv glog-0.3.4/ glog 24 | rm -fr v0.3.4.tar.gz 25 | cd glog 26 | ./configure --prefix=$PWD 27 | make -j 28 | make install 29 | cd .. 30 | 31 | 32 | # zeromq 33 | git clone https://github.com/zeromq/libzmq.git 34 | mv libzmq zeromq 35 | cd zeromq 36 | ./autogen.sh 37 | ./configure --prefix=$PWD 38 | make -j 39 | make install 40 | cd .. 41 | 42 | 43 | #gflags 44 | wget https://github.com/gflags/gflags/archive/v2.0.tar.gz 45 | tar -xzf v2.0.tar.gz 46 | rm v2.0.tar.gz 47 | mv gflags-2.0/ gflags 48 | cd gflags 49 | ./configure --prefix=$PWD 50 | make -j 51 | make install 52 | cd .. 53 | 54 | 55 | -------------------------------------------------------------------------------- /src/Makefile: -------------------------------------------------------------------------------- 1 | # 2 | # This makefile system follows the structuring conventions 3 | # recommended by Peter Miller in his excellent paper: 4 | # 5 | # Recursive Make Considered Harmful 6 | # http://aegis.sourceforge.net/auug97.pdf 7 | # 8 | 9 | # We create a listing of the root directories for access into 10 | OBJDIR := ../obj 11 | BINDIR := ../bin 12 | EXTDIR := ../ext 13 | SRCDIR := ../src 14 | LOGDIR := ../logs 15 | 16 | # Next, we define the external libraries we use and their directories 17 | PROTOB := $(EXTDIR)/protobuf 18 | GLOG := $(EXTDIR)/glog 19 | GFLAGS := $(EXTDIR)/gflags 20 | ZEROMQ := $(EXTDIR)/zeromq 21 | 22 | # Executable for protocol buffers 23 | PROTOCEXE := $(PROTOB)/src/.libs/protoc 24 | 25 | # '$(V)' controls whether the lab makefiles print verbose commands (the 26 | # actual shell commands run by Make), as well as the "overview" commands 27 | # (such as '+ cc lib/readline.c'). 28 | # 29 | # For overview commands only, the line should read 'V = @'. 30 | # For overview and verbose commands, the line should read 'V ='. 31 | V = @ 32 | 33 | # Set the compiler and compile-time loaded libraries 34 | CXX := g++ 35 | LDLIBRARYPATH := LD_LIBRARY_PATH=$(ZEROMQ)/src/.libs:$(PROTOB)/src/.libs:$(GLOG)/.libs:$(GFLAGS)/.libs 36 | 37 | #----------------------------------------------- 38 | # OPT MODES 39 | PROD := -O2 -DNDEBUG # (A) Production use (optimized mode) 40 | DEBUG := -g2 # (B) Debug mode, w/ line-level debug symbols 41 | PROFILING := -O2 -g2 -DNDEBUG # (C) opt, but w/debug symbols 42 | 43 | OPT_MODE := $(PROD) 44 | 45 | # Set the flags for C++ to compile with (namely where to look for external 46 | # libraries) and the linker libraries (again to look in the ext/ library) 47 | CXXFLAGS := -O2 -DNDEBUG -DNDEBUG -MD -I$(SRCDIR) -I$(OBJDIR) \ 48 | -I$(ZEROMQ)/include \ 49 | -I$(PROTOB)/src \ 50 | -I$(GLOG)/include \ 51 | -I$(GFLAGS)/include \ 52 | -Wall -Wno-strict-aliasing -std=c++0x -fno-omit-frame-pointer 53 | 54 | LDFLAGS := -lglog -L$(GLOG)/.libs \ 55 | -lgflags -L$(GFLAGS)/.libs \ 56 | -lprotobuf -L$(PROTOB)/src/.libs \ 57 | -lrt -lreadline -lpthread 58 | 59 | ZMQLDFLAGS := -lzmq -L$(ZEROMQ)/src/.libs 60 | 61 | # Lists that the */Makefile.inc makefile fragments will add to 62 | OBJDIRS := 63 | TESTS := 64 | 65 | # Make sure that 'all' is the first target 66 | all: 67 | @mkdir -p $(LOGDIR) 68 | 69 | # Makefile template so that the makefrags are far less redundant 70 | MAKEFILE_TEMPLATE := Makefile.template 71 | 72 | # python obj file used by some components 73 | PYTHON_OBJS := /usr/lib64/libpython2.7.so 74 | 75 | # Makefile fragments for specific submodules 76 | include proto/Makefile.inc 77 | include common/Makefile.inc 78 | include machine/Makefile.inc 79 | include log/Makefile.inc 80 | include backend/Makefile.inc 81 | include applications/Makefile.inc 82 | include scheduler/Makefile.inc 83 | include scripts/Makefile.inc 84 | 85 | test: $(TESTS) 86 | @mkdir -p $(LOGDIR) 87 | clean: 88 | rm -rf $(OBJDIR) $(BINDIR) $(LOGDIR) 89 | 90 | # This magic automatically generates makefile dependencies 91 | # for header files included from C source files we compile, 92 | # and keeps those dependencies up-to-date every time we recompile. 93 | # See 'mergedep.pl' for more information. 94 | $(OBJDIR)/.deps: $(foreach dir, $(OBJDIRS), $(wildcard $(OBJDIR)/$(dir)/*.d)) 95 | @mkdir -p $(@D) 96 | @cat $^ /dev/null > $@ 97 | # @$(PERL) mergedep.pl $@ $^ 98 | -include $(OBJDIR)/.deps 99 | 100 | always: 101 | 102 | # Eliminate default suffix rules 103 | .SUFFIXES: 104 | 105 | # make it so that no intermediate .o files are ever deleted 106 | .PRECIOUS: $(foreach dir, $(OBJDIRS), $(OBJDIR)/$(dir)/%.o) \ 107 | $(foreach dir, $(OBJDIRS), $(OBJDIR)/$(dir)/%.pb.cc) \ 108 | $(foreach dir, $(OBJDIRS), $(OBJDIR)/$(dir)/%.pb.h) 109 | 110 | .PHONY: all always clean test 111 | -------------------------------------------------------------------------------- /src/Makefile.default: -------------------------------------------------------------------------------- 1 | # 2 | # This makefile system follows the structuring conventions 3 | # recommended by Peter Miller in his excellent paper: 4 | # 5 | # Recursive Make Considered Harmful 6 | # http://aegis.sourceforge.net/auug97.pdf 7 | # 8 | 9 | # We create a listing of the root directories for access into 10 | OBJDIR := ../obj 11 | BINDIR := ../bin 12 | EXTDIR := ../ext 13 | SRCDIR := ../src 14 | LOGDIR := ../logs 15 | 16 | # Next, we define the external libraries we use and their directories 17 | PROTOB := $(EXTDIR)/protobuf 18 | GLOG := $(EXTDIR)/glog 19 | GFLAGS := $(EXTDIR)/gflags 20 | ZEROMQ := $(EXTDIR)/zeromq 21 | 22 | # Executable for protocol buffers 23 | PROTOCEXE := $(PROTOB)/src/.libs/protoc 24 | 25 | # '$(V)' controls whether the lab makefiles print verbose commands (the 26 | # actual shell commands run by Make), as well as the "overview" commands 27 | # (such as '+ cc lib/readline.c'). 28 | # 29 | # For overview commands only, the line should read 'V = @'. 30 | # For overview and verbose commands, the line should read 'V ='. 31 | V = @ 32 | 33 | # Set the compiler and compile-time loaded libraries 34 | CXX := g++ 35 | LDLIBRARYPATH := LD_LIBRARY_PATH=$(ZEROMQ)/src/.libs:$(PROTOB)/src/.libs:$(GLOG)/.libs:$(GFLAGS)/.libs 36 | 37 | #----------------------------------------------- 38 | # OPT MODES 39 | PROD := -O2 -DNDEBUG # (A) Production use (optimized mode) 40 | DEBUG := -g2 # (B) Debug mode, w/ line-level debug symbols 41 | PROFILING := -O2 -g2 -DNDEBUG # (C) opt, but w/debug symbols 42 | 43 | OPT_MODE := $(PROD) 44 | 45 | # Set the flags for C++ to compile with (namely where to look for external 46 | # libraries) and the linker libraries (again to look in the ext/ library) 47 | CXXFLAGS := -O2 -DNDEBUG -MD -I$(SRCDIR) -I$(OBJDIR) \ 48 | -I$(ZEROMQ)/include \ 49 | -I$(PROTOB)/src \ 50 | -I$(GLOG)/include \ 51 | -I$(GFLAGS)/include \ 52 | -Wall -Wno-strict-aliasing -std=c++0x -fno-omit-frame-pointer 53 | 54 | LDFLAGS := -lglog -L$(GLOG)/.libs \ 55 | -lgflags -L$(GFLAGS)/.libs \ 56 | -lprotobuf -L$(PROTOB)/src/.libs \ 57 | -lrt -lreadline -lpthread 58 | 59 | ZMQLDFLAGS := -lzmq -L$(ZEROMQ)/src/.libs 60 | 61 | # Lists that the */Makefile.inc makefile fragments will add to 62 | OBJDIRS := 63 | TESTS := 64 | 65 | # Make sure that 'all' is the first target 66 | all: 67 | @mkdir -p $(LOGDIR) 68 | 69 | # Makefile template so that the makefrags are far less redundant 70 | MAKEFILE_TEMPLATE := Makefile.template 71 | 72 | # python obj file used by some components 73 | PYTHON_OBJS := /usr/lib64/libpython2.7.so 74 | 75 | # Makefile fragments for specific submodules 76 | include proto/Makefile.inc 77 | include common/Makefile.inc 78 | include machine/Makefile.inc 79 | include log/Makefile.inc 80 | include backend/Makefile.inc 81 | include applications/Makefile.inc 82 | include scheduler/Makefile.inc 83 | include scripts/Makefile.inc 84 | 85 | test: $(TESTS) 86 | @mkdir -p $(LOGDIR) 87 | clean: 88 | rm -rf $(OBJDIR) $(BINDIR) $(LOGDIR) 89 | 90 | # This magic automatically generates makefile dependencies 91 | # for header files included from C source files we compile, 92 | # and keeps those dependencies up-to-date every time we recompile. 93 | # See 'mergedep.pl' for more information. 94 | $(OBJDIR)/.deps: $(foreach dir, $(OBJDIRS), $(wildcard $(OBJDIR)/$(dir)/*.d)) 95 | @mkdir -p $(@D) 96 | @cat $^ /dev/null > $@ 97 | # @$(PERL) mergedep.pl $@ $^ 98 | -include $(OBJDIR)/.deps 99 | 100 | always: 101 | 102 | # Eliminate default suffix rules 103 | .SUFFIXES: 104 | 105 | # make it so that no intermediate .o files are ever deleted 106 | .PRECIOUS: $(foreach dir, $(OBJDIRS), $(OBJDIR)/$(dir)/%.o) \ 107 | $(foreach dir, $(OBJDIRS), $(OBJDIR)/$(dir)/%.pb.cc) \ 108 | $(foreach dir, $(OBJDIRS), $(OBJDIR)/$(dir)/%.pb.h) 109 | 110 | .PHONY: all always clean test 111 | -------------------------------------------------------------------------------- /src/Makefile.template: -------------------------------------------------------------------------------- 1 | OBJDIRS += $(LC_DIR) 2 | 3 | $(UC_DIR)_PROTO_DEFS := $(PROTOS) 4 | $(UC_DIR)_SRCS := $(SRCS) 5 | $(UC_DIR)_EXES := $(EXES) 6 | $(UC_DIR)_TEST_SRCS := $(TEST) 7 | 8 | # proto stuff 9 | $(UC_DIR)_PROTO_SRCS := $(patsubst %.proto, $(OBJDIR)/%.pb.cc, $($(UC_DIR)_PROTO_DEFS)) 10 | $(UC_DIR)_PROTO_OBJS := $(patsubst %.c, %.o,\ 11 | $(patsubst %.cpp, %.o,\ 12 | $(patsubst %.cc, %.o, $($(UC_DIR)_PROTO_SRCS)))) 13 | 14 | $(OBJDIR)/$(LC_DIR)/%.pb.cc $(OBJDIR)/$(LC_DIR)/%.pb.h: $(LC_DIR)/%.proto 15 | @echo + protoc $@ 16 | @mkdir -p $(@D) 17 | $(V)$(LDLIBRARYPATH) $(PROTOCEXE) --proto_path=$(patsubst %/,%,$(dir $<)) \ 18 | --cpp_out=$(OBJDIR)/$(patsubst %/,%,$(dir $<)) $< 19 | 20 | # Build files only if they exist. 21 | $(UC_DIR)_OBJS := $(patsubst %.c, $(OBJDIR)/%.o,\ 22 | $(patsubst %.cpp, $(OBJDIR)/%.o,\ 23 | $(patsubst %.cc, $(OBJDIR)/%.o, $($(UC_DIR)_SRCS)))) 24 | $(UC_DIR)_OBJS += $($(UC_DIR)_PROTO_OBJS) 25 | 26 | ifneq ($($(UC_DIR)_EXES),) 27 | $(UC_DIR)_PROG := $(patsubst $(LC_DIR)/%.cc, $(BINDIR)/$(LC_DIR)/%, $($(UC_DIR)_EXES)) 28 | endif 29 | 30 | $(UC_DIR)_TEST_OBJS := $(patsubst %.cc, $(OBJDIR)/%.o, $($(UC_DIR)_TEST_SRCS)) 31 | $(UC_DIR)_TESTS := $(patsubst %.cc, $(BINDIR)/%, $($(UC_DIR)_TEST_SRCS)) 32 | 33 | TESTS += test-$(LC_DIR) 34 | 35 | all: $(LC_DIR) 36 | $(LC_DIR): $($(UC_DIR)_PROG) $($(UC_DIR)_OBJS) $($(UC_DIR)_TESTS) 37 | 38 | $(OBJDIR)/$(LC_DIR)/%.pb.o: $(OBJDIR)/$(LC_DIR)/%.pb.cc $(DEPS) 39 | @echo + cxx $@ 40 | @mkdir -p $(@D) 41 | $(V)$(CXX) $(CXXFLAGS) -o $@ -c $< 42 | 43 | $(OBJDIR)/$(LC_DIR)/%.o: $(LC_DIR)/%.cc $($(UC_DIR)_PROTO_SRCS) $(DEPS) 44 | @echo + cxx $@ 45 | @mkdir -p $(@D) 46 | $(V)$(CXX) $(CXXFLAGS) -o $@ -c $< 47 | 48 | $(BINDIR)/$(LC_DIR)/%_test: $(OBJDIR)/$(LC_DIR)/%_test.o $($(UC_DIR)_OBJS) $(DEPS) 49 | @echo + ld $@ 50 | @mkdir -p $(@D) 51 | $(V)$(CXX) -o $@ $^ $(LDFLAGS) $(ZMQLDFLAGS) 52 | 53 | # Compiled CPP executables 54 | ifneq ($($(UC_DIR)_EXES),) 55 | $(BINDIR)/$(LC_DIR)/%: $(LC_DIR)/%.cc $(DEPS) $($(UC_DIR)_OBJS) 56 | @echo + ld $@ 57 | @mkdir -p $(@D) 58 | $(V)$(CXX) -o $@ $^ $(CXXFLAGS) $(LDFLAGS) $(ZMQLDFLAGS) 59 | endif 60 | 61 | .PHONY: $(LC_DIR) test-$(LC_DIR) 62 | 63 | 64 | -------------------------------------------------------------------------------- /src/applications/Makefile.inc: -------------------------------------------------------------------------------- 1 | # Create template specific variables 2 | UC_DIR := APPLICATIONS 3 | LC_DIR := applications 4 | 5 | SRCS := applications/microbenchmark.cc \ 6 | applications/tpcc.cc 7 | 8 | EXES := 9 | 10 | DEPS := $(PROTO_OBJS) $(COMMON_OBJS) $(BACKEND_OBJS) 11 | 12 | # Link the template to avoid redundancy 13 | include $(MAKEFILE_TEMPLATE) 14 | -------------------------------------------------------------------------------- /src/applications/application.h: -------------------------------------------------------------------------------- 1 | // Author: Kun Ren 2 | // 3 | // The Application abstract class 4 | // 5 | // Application execution logic in the system is coded into 6 | 7 | #ifndef _DB_APPLICATIONS_APPLICATION_H_ 8 | #define _DB_APPLICATIONS_APPLICATION_H_ 9 | 10 | #include 11 | 12 | #include "common/types.h" 13 | 14 | using std::string; 15 | 16 | class ClusterConfig; 17 | class Storage; 18 | class StorageManager; 19 | class TxnProto; 20 | 21 | enum TxnStatus { 22 | SUCCESS = 0, 23 | FAILURE = 1, 24 | REDO = 2, 25 | }; 26 | 27 | class Application { 28 | public: 29 | virtual ~Application() {} 30 | 31 | // Load generation. 32 | virtual TxnProto* NewTxn(int64 txn_id, int txn_type, string args, 33 | ClusterConfig* config) const = 0; 34 | 35 | // Execute a transaction's application logic given the input 'txn'. 36 | virtual int Execute(TxnProto* txn, StorageManager* storage) const = 0; 37 | 38 | // Storage initialization method. 39 | virtual void InitializeStorage(Storage* storage, 40 | ClusterConfig* conf) const = 0; 41 | }; 42 | 43 | #endif // _DB_APPLICATIONS_APPLICATION_H_ 44 | -------------------------------------------------------------------------------- /src/applications/microbenchmark.cc: -------------------------------------------------------------------------------- 1 | // Author: Kun Ren 2 | // 3 | 4 | #include "applications/microbenchmark.h" 5 | 6 | #include 7 | 8 | #include "backend/storage.h" 9 | #include "backend/storage_manager.h" 10 | #include "common/utils.h" 11 | #include "proto/txn.pb.h" 12 | 13 | 14 | // Fills '*keys' with num_keys unique ints k where 15 | // 'key_start' <= k < 'key_limit', and k == part (mod nparts). 16 | // Requires: key_start % nparts == 0 17 | void Microbenchmark::GetRandomKeys(set* keys, uint32 num_keys, uint64 key_start, 18 | uint64 key_limit, uint64 part) { 19 | CHECK(key_start % nparts == 0); 20 | keys->clear(); 21 | for (uint32 i = 0; i < num_keys; i++) { 22 | // Find a key not already in '*keys'. 23 | uint64 key; 24 | do { 25 | key = key_start + part + nparts * (rand() % ((key_limit - key_start)/nparts)); 26 | } while (keys->count(key)); 27 | keys->insert(key); 28 | } 29 | } 30 | 31 | // Fills '*keys' with num_keys unique ints k where 32 | // 'key_start' <= k < 'key_limit', and k == part (mod nparts), keys's master == replica 33 | // Requires: key_start % nparts == 0 34 | void Microbenchmark::GetRandomKeysReplica(set* keys, uint32 num_keys, uint64 key_start, 35 | uint64 key_limit, uint64 part, uint32 replica) { 36 | CHECK(key_start % (nparts*replica_size) == 0); 37 | keys->clear(); 38 | 39 | for (uint32 i = 0; i < num_keys; i++) { 40 | // Find a key not already in '*keys'. 41 | uint64 key; 42 | uint64 order = rand() % ((key_limit - key_start)/(nparts*replica_size)); 43 | key = key_start + part + nparts * (order * replica_size + replica); 44 | 45 | while (keys->count(key)) { 46 | order = rand() % ((key_limit - key_start)/(nparts*replica_size)); 47 | key = key_start + part + nparts * (order * replica_size + replica); 48 | } 49 | 50 | keys->insert(key); 51 | } 52 | } 53 | 54 | 55 | //--------- Create a single-partition transaction ------------------------- 56 | TxnProto* Microbenchmark::MicroTxnSP(int64 txn_id, uint64 part) { 57 | // Create the new transaction object 58 | TxnProto* txn = new TxnProto(); 59 | 60 | // Set the transaction's standard attributes 61 | txn->set_txn_id(txn_id); 62 | txn->set_txn_type(MICROTXN_SP); 63 | 64 | // Add two hot keys to read/write set. 65 | uint64 hotkey1 = part + nparts * (rand() % hot_records); 66 | 67 | KeyEntry* key_entry = txn->add_read_write_set(); 68 | key_entry->set_key(IntToString(hotkey1)); 69 | key_entry->set_master(0); 70 | key_entry->set_counter(0); 71 | 72 | uint64 hotkey2 = part + nparts * (rand() % hot_records); 73 | while (hotkey2 == hotkey1) { 74 | hotkey2 = part + nparts * (rand() % hot_records); 75 | }; 76 | 77 | key_entry = txn->add_read_write_set(); 78 | key_entry->set_key(IntToString(hotkey2)); 79 | key_entry->set_master(0); 80 | key_entry->set_counter(0); 81 | 82 | // Insert set of kRWSetSize - 1 random cold keys from specified partition into 83 | // read/write set. 84 | set keys; 85 | GetRandomKeys(&keys, 86 | kRWSetSize - 2, 87 | nparts * hot_records, 88 | nparts * kDBSize, 89 | part); 90 | for (set::iterator it = keys.begin(); it != keys.end(); ++it) { 91 | key_entry = txn->add_read_write_set(); 92 | key_entry->set_key(IntToString(*it)); 93 | key_entry->set_master(0); 94 | key_entry->set_counter(0); 95 | } 96 | 97 | return txn; 98 | } 99 | 100 | //----------- Create a multi-partition transaction ------------------------- 101 | TxnProto* Microbenchmark::MicroTxnMP(int64 txn_id, uint64 part1, uint64 part2) { 102 | CHECK(part1 != part2 || nparts == 1); 103 | // Create the new transaction object 104 | TxnProto* txn = new TxnProto(); 105 | 106 | // Set the transaction's standard attributes 107 | txn->set_txn_id(txn_id); 108 | txn->set_txn_type(MICROTXN_MP); 109 | 110 | // Add two hot keys to read/write set---one in each partition. 111 | uint64 hotkey1 = part1 + nparts * (rand() % hot_records); 112 | uint64 hotkey2 = part2 + nparts * (rand() % hot_records); 113 | 114 | KeyEntry* key_entry = txn->add_read_write_set(); 115 | key_entry->set_key(IntToString(hotkey1)); 116 | key_entry->set_master(0); 117 | key_entry->set_counter(0); 118 | 119 | key_entry = txn->add_read_write_set(); 120 | key_entry->set_key(IntToString(hotkey2)); 121 | key_entry->set_master(0); 122 | key_entry->set_counter(0); 123 | 124 | // Insert set of kRWSetSize/2 - 1 random cold keys from each partition into 125 | // read/write set. 126 | set keys; 127 | GetRandomKeys(&keys, 128 | kRWSetSize/2 - 1, 129 | nparts * hot_records, 130 | nparts * kDBSize, 131 | part1); 132 | for (set::iterator it = keys.begin(); it != keys.end(); ++it) { 133 | key_entry = txn->add_read_write_set(); 134 | key_entry->set_key(IntToString(*it)); 135 | key_entry->set_master(0); 136 | key_entry->set_counter(0); 137 | } 138 | 139 | GetRandomKeys(&keys, 140 | kRWSetSize/2 - 1, 141 | nparts * hot_records, 142 | nparts * kDBSize, 143 | part2); 144 | for (set::iterator it = keys.begin(); it != keys.end(); ++it) { 145 | key_entry = txn->add_read_write_set(); 146 | key_entry->set_key(IntToString(*it)); 147 | key_entry->set_master(0); 148 | key_entry->set_counter(0); 149 | } 150 | 151 | return txn; 152 | } 153 | 154 | //------------- Create a single-replica single-partition transaction------------ 155 | TxnProto* Microbenchmark::MicroTxnSRSP(int64 txn_id, uint64 part, uint32 replica) { 156 | // Create the new transaction object 157 | TxnProto* txn = new TxnProto(); 158 | 159 | // Set the transaction's standard attributes 160 | txn->set_txn_id(txn_id); 161 | txn->set_txn_type(MICROTXN_SP); 162 | 163 | //if (replica == 0) 164 | //LOG(ERROR) << ": In Microbenchmark::MicroTxnSRSP: 1"; 165 | 166 | // Add two hot keys to read/write set. 167 | uint64 hotkey_order1 = (rand() % (hot_records/replica_size)) * replica_size + replica; 168 | 169 | uint64 hotkey_order2 = (rand() % (hot_records/replica_size)) * replica_size + replica; 170 | while (hotkey_order2 == hotkey_order1) { 171 | hotkey_order2 = (rand() % (hot_records/replica_size)) * replica_size + replica; 172 | }; 173 | 174 | 175 | //if (replica == 0) 176 | //LOG(ERROR) << ": In Microbenchmark::MicroTxnSRSP: 2"; 177 | 178 | uint64 hotkey1 = part + nparts * hotkey_order1; 179 | uint64 hotkey2 = part + nparts * hotkey_order2; 180 | 181 | KeyEntry* key_entry = txn->add_read_write_set(); 182 | key_entry->set_key(IntToString(hotkey1)); 183 | key_entry->set_master(replica); 184 | key_entry->set_counter(0); 185 | 186 | key_entry = txn->add_read_write_set(); 187 | key_entry->set_key(IntToString(hotkey2)); 188 | key_entry->set_master(replica); 189 | key_entry->set_counter(0); 190 | 191 | // Insert set of kRWSetSize - 1 random cold keys from specified partition into 192 | // read/write set. 193 | uint64 key_start = nparts * hot_records; 194 | if (key_start % (replica_size*nparts) != 0) { 195 | key_start = key_start + (replica_size*nparts - (key_start % (replica_size*nparts))); 196 | } 197 | 198 | set keys; 199 | GetRandomKeysReplica(&keys, 200 | kRWSetSize - 2, 201 | key_start, 202 | nparts * kDBSize, 203 | part, 204 | replica); 205 | for (set::iterator it = keys.begin(); it != keys.end(); ++it) { 206 | key_entry = txn->add_read_write_set(); 207 | key_entry->set_key(IntToString(*it)); 208 | key_entry->set_master(replica); 209 | key_entry->set_counter(0); 210 | } 211 | 212 | //if (replica == 0) 213 | //LOG(ERROR) << ": In Microbenchmark::MicroTxnSRSP: 3"; 214 | 215 | return txn; 216 | } 217 | 218 | //------------- Create a single-replica multi-partition transaction------------ 219 | TxnProto* Microbenchmark::MicroTxnSRMP(int64 txn_id, uint64 part1, uint64 part2, uint32 replica) { 220 | CHECK(part1 != part2 || nparts == 1); 221 | // Create the new transaction object 222 | TxnProto* txn = new TxnProto(); 223 | 224 | // Set the transaction's standard attributes 225 | txn->set_txn_id(txn_id); 226 | txn->set_txn_type(MICROTXN_MP); 227 | 228 | // Add two hot keys to read/write set---one in each partition. 229 | uint64 hotkey_order1 = (rand() % (hot_records/replica_size)) * replica_size + replica; 230 | uint64 hotkey_order2 = (rand() % (hot_records/replica_size)) * replica_size + replica; 231 | 232 | 233 | uint64 hotkey1 = part1 + nparts * hotkey_order1; 234 | uint64 hotkey2 = part2 + nparts * hotkey_order2; 235 | 236 | KeyEntry* key_entry = txn->add_read_write_set(); 237 | key_entry->set_key(IntToString(hotkey1)); 238 | key_entry->set_master(replica); 239 | key_entry->set_counter(0); 240 | 241 | key_entry = txn->add_read_write_set(); 242 | key_entry->set_key(IntToString(hotkey2)); 243 | key_entry->set_master(replica); 244 | key_entry->set_counter(0); 245 | 246 | // Insert set of kRWSetSize/2 - 1 random cold keys from each partition into 247 | // read/write set. 248 | uint64 key_start = nparts * hot_records; 249 | if (key_start % (replica_size*nparts) != 0) { 250 | key_start = key_start + (replica_size*nparts - (key_start % (replica_size*nparts))); 251 | } 252 | 253 | set keys; 254 | GetRandomKeysReplica(&keys, 255 | kRWSetSize/2 - 1, 256 | key_start, 257 | nparts * kDBSize, 258 | part1, 259 | replica); 260 | for (set::iterator it = keys.begin(); it != keys.end(); ++it) { 261 | key_entry = txn->add_read_write_set(); 262 | key_entry->set_key(IntToString(*it)); 263 | key_entry->set_master(replica); 264 | key_entry->set_counter(0); 265 | } 266 | 267 | GetRandomKeysReplica(&keys, 268 | kRWSetSize/2 - 1, 269 | key_start, 270 | nparts * kDBSize, 271 | part2, 272 | replica); 273 | for (set::iterator it = keys.begin(); it != keys.end(); ++it) { 274 | key_entry = txn->add_read_write_set(); 275 | key_entry->set_key(IntToString(*it)); 276 | key_entry->set_master(replica); 277 | key_entry->set_counter(0); 278 | } 279 | 280 | return txn; 281 | } 282 | 283 | //------------- Create a multi-replica single-partition transaction------------ 284 | TxnProto* Microbenchmark::MicroTxnMRSP(int64 txn_id, uint64 part, uint32 replica1, uint32 replica2) { 285 | CHECK(replica1 != replica2); 286 | // Create the new transaction object 287 | TxnProto* txn = new TxnProto(); 288 | 289 | // Set the transaction's standard attributes 290 | txn->set_txn_id(txn_id); 291 | txn->set_txn_type(MICROTXN_SP); 292 | 293 | if (replica1 > replica2) { 294 | uint32 tmp = replica1; 295 | replica1 = replica2; 296 | replica2 = tmp; 297 | } 298 | 299 | if (replica1 != 0 && replica2 != 0) { 300 | txn->set_fake_txn(true); 301 | } 302 | 303 | // Add two hot keys to read/write set. 304 | uint64 hotkey_order1 = (rand() % (hot_records/replica_size)) * replica_size + replica1; 305 | uint64 hotkey_order2 = (rand() % (hot_records/replica_size)) * replica_size + replica2; 306 | 307 | uint64 hotkey1 = part + nparts * hotkey_order1; 308 | uint64 hotkey2 = part + nparts * hotkey_order2; 309 | 310 | KeyEntry* key_entry = txn->add_read_write_set(); 311 | key_entry->set_key(IntToString(hotkey1)); 312 | key_entry->set_master(replica1); 313 | key_entry->set_counter(0); 314 | 315 | key_entry = txn->add_read_write_set(); 316 | key_entry->set_key(IntToString(hotkey2)); 317 | key_entry->set_master(replica2); 318 | key_entry->set_counter(0); 319 | 320 | // Insert set of kRWSetSize/2 - 1 random cold keys from specified replica/partition into 321 | // read/write set. 322 | 323 | uint64 key_start = nparts * hot_records; 324 | if (key_start % (replica_size*nparts) != 0) { 325 | key_start = key_start + (replica_size*nparts - (key_start % (replica_size*nparts))); 326 | } 327 | 328 | set keys; 329 | GetRandomKeysReplica(&keys, 330 | kRWSetSize/2 - 1, 331 | key_start, 332 | nparts * kDBSize, 333 | part, 334 | replica1); 335 | for (set::iterator it = keys.begin(); it != keys.end(); ++it) { 336 | key_entry = txn->add_read_write_set(); 337 | key_entry->set_key(IntToString(*it)); 338 | key_entry->set_master(replica1); 339 | key_entry->set_counter(0); 340 | } 341 | 342 | // Insert set of kRWSetSize/2 - 1 random cold keys from specified replica/partition into 343 | // read/write set. 344 | GetRandomKeysReplica(&keys, 345 | kRWSetSize/2 - 1, 346 | key_start, 347 | nparts * kDBSize, 348 | part, 349 | replica2); 350 | for (set::iterator it = keys.begin(); it != keys.end(); ++it) { 351 | key_entry = txn->add_read_write_set(); 352 | key_entry->set_key(IntToString(*it)); 353 | key_entry->set_master(replica2); 354 | key_entry->set_counter(0); 355 | } 356 | 357 | return txn; 358 | } 359 | 360 | //------------- Create a multi-replica multi-partition transaction------------ 361 | TxnProto* Microbenchmark::MicroTxnMRMP(int64 txn_id, uint64 part1, uint64 part2, uint32 replica1, uint32 replica2) { 362 | CHECK(part1 != part2 || nparts == 1); 363 | // Create the new transaction object 364 | TxnProto* txn = new TxnProto(); 365 | 366 | // Set the transaction's standard attributes 367 | txn->set_txn_id(txn_id); 368 | txn->set_txn_type(MICROTXN_MP); 369 | 370 | if (replica1 > replica2) { 371 | uint32 tmp = replica1; 372 | replica1 = replica2; 373 | replica2 = tmp; 374 | 375 | uint64 tmp2 = part1; 376 | part1 = part2; 377 | part2 = tmp2; 378 | } 379 | 380 | if (replica1 != 0 && replica2 != 0) { 381 | txn->set_fake_txn(true); 382 | } 383 | 384 | // Add two hot keys to read/write set---one in each partition. 385 | uint64 hotkey_order1 = (rand() % (hot_records/replica_size)) * replica_size + replica1; 386 | uint64 hotkey_order2 = (rand() % (hot_records/replica_size)) * replica_size + replica2; 387 | 388 | uint64 hotkey1 = part1 + nparts * hotkey_order1; 389 | uint64 hotkey2 = part2 + nparts * hotkey_order2; 390 | 391 | KeyEntry* key_entry = txn->add_read_write_set(); 392 | key_entry->set_key(IntToString(hotkey1)); 393 | key_entry->set_master(replica1); 394 | key_entry->set_counter(0); 395 | 396 | key_entry = txn->add_read_write_set(); 397 | key_entry->set_key(IntToString(hotkey2)); 398 | key_entry->set_master(replica2); 399 | key_entry->set_counter(0); 400 | 401 | // Insert set of kRWSetSize/2 - 1 random cold keys from each replica/partition into 402 | // read/write set. 403 | 404 | uint64 key_start = nparts * hot_records; 405 | if (key_start % (replica_size*nparts) != 0) { 406 | key_start = key_start + (replica_size*nparts - (key_start % (replica_size*nparts))); 407 | } 408 | 409 | 410 | set keys; 411 | GetRandomKeysReplica(&keys, 412 | kRWSetSize/2 - 1, 413 | key_start, 414 | nparts * kDBSize, 415 | part1, 416 | replica1); 417 | for (set::iterator it = keys.begin(); it != keys.end(); ++it) { 418 | key_entry = txn->add_read_write_set(); 419 | key_entry->set_key(IntToString(*it)); 420 | key_entry->set_master(replica1); 421 | key_entry->set_counter(0); 422 | } 423 | 424 | // Insert set of kRWSetSize/2 - 1 random cold keys from each replica/partition into 425 | // read/write set. 426 | GetRandomKeysReplica(&keys, 427 | kRWSetSize/2 - 1, 428 | key_start, 429 | nparts * kDBSize, 430 | part2, 431 | replica2); 432 | for (set::iterator it = keys.begin(); it != keys.end(); ++it) { 433 | key_entry = txn->add_read_write_set(); 434 | key_entry->set_key(IntToString(*it)); 435 | key_entry->set_master(replica2); 436 | key_entry->set_counter(0); 437 | } 438 | 439 | return txn; 440 | } 441 | 442 | 443 | // The load generator can be called externally to return a transaction proto 444 | // containing a new type of transaction. 445 | TxnProto* Microbenchmark::NewTxn(int64 txn_id, int txn_type, 446 | string args, ClusterConfig* config) const { 447 | return NULL; 448 | } 449 | 450 | int Microbenchmark::Execute(TxnProto* txn, StorageManager* storage) const { 451 | // Remaster txn 452 | if (txn->remaster_txn() == true) { 453 | LOG(ERROR) <txn_id(); 454 | KeyEntry key_entry = txn->read_write_set(0); 455 | Record* val = storage->ReadObject(key_entry.key()); 456 | 457 | CHECK(key_entry.master() == val->master); 458 | CHECK(key_entry.counter() == val->counter); 459 | 460 | val->master = txn->remaster_to(); 461 | val->counter = key_entry.counter() + 1; 462 | 463 | if (local_replica_ == txn->remaster_from()) { 464 | val->remastering = false; 465 | } 466 | 467 | return 0; 468 | } 469 | 470 | // Normal txns, read the record and do some computations 471 | double factor = 1.0; 472 | uint32 txn_type = txn->txn_type(); 473 | if (txn_type == MICROTXN_MP || (txn_type == MICROTXN_SP && txn->involved_replicas_size() > 1)) { 474 | factor = 2.0; 475 | } 476 | double execution_start = GetTime(); 477 | 478 | 479 | for (uint32 i = 0; i < kRWSetSize; i++) { 480 | KeyEntry key_entry = txn->read_write_set(i); 481 | Record* val = storage->ReadObject(key_entry.key()); 482 | // Not necessary since storage already has a pointer to val. 483 | // storage->PutObject(txn->read_write_set(i), val); 484 | 485 | // Check whether we need to remaster this record 486 | if (storage->GetMode() == 2 && local_replica_ == val->master && val->remastering == false) { 487 | val->access_pattern[txn->client_replica()] = val->access_pattern[txn->client_replica()] + 1; 488 | 489 | if (txn->client_replica() != local_replica_ && val->access_pattern[txn->client_replica()]/(LAST_N_TOUCH*1.0) > ACCESS_PATTERN_THRESHOLD) { 490 | // Reach the threadhold, do the remaster 491 | val->remastering = true; 492 | 493 | // Create the remaster transction and sent to local sequencer 494 | TxnProto* remaster_txn = new TxnProto(); 495 | 496 | remaster_txn->set_txn_id(config_->GetGUID()); 497 | KeyEntry* remaster_key_entry = remaster_txn->add_read_write_set(); 498 | remaster_key_entry->set_key(key_entry.key()); 499 | remaster_key_entry->set_master(val->master); 500 | remaster_key_entry->set_counter(val->counter); 501 | 502 | remaster_txn->set_remaster_txn(true); 503 | remaster_txn->set_remaster_from(local_replica_); 504 | remaster_txn->set_remaster_to(txn->client_replica()); 505 | 506 | remaster_txn->add_involved_replicas(local_replica_); 507 | 508 | string txn_string; 509 | remaster_txn->SerializeToString(&txn_string); 510 | 511 | MessageProto txn_message; 512 | txn_message.set_destination_channel("sequencer_txn_receive_"); 513 | txn_message.set_type(MessageProto::TXN_FORWORD); 514 | txn_message.set_destination_node(config_->local_node_id()); 515 | txn_message.add_data(txn_string); 516 | 517 | connection_->Send(txn_message); 518 | LOG(ERROR) <txn_id(); 519 | } 520 | 521 | 522 | if (++val->access_cnt > LAST_N_TOUCH) { 523 | for (uint32 j = 0; j < REPLICA_SIZE; j++) { 524 | val->access_pattern[j] = 0; 525 | } 526 | val->access_cnt = 0; 527 | } 528 | 529 | } 530 | 531 | 532 | for (int j = 0; j < 8; j++) { 533 | if ((val->value)[j] + 1 > 'z') { 534 | (val->value)[j] = 'a'; 535 | } else { 536 | (val->value)[j] = (val->value)[j] + 1; 537 | } 538 | } 539 | 540 | } 541 | 542 | // The following code is for microbenchmark "long" transaction, uncomment it if for "long" transaction 543 | while (GetTime() - execution_start < 0.00012/factor) { 544 | int x = 1; 545 | for(int i = 0; i < 10000; i++) { 546 | x = x+10; 547 | x = x-2; 548 | } 549 | } 550 | 551 | return 0; 552 | } 553 | 554 | void Microbenchmark::InitializeStorage(Storage* storage, ClusterConfig* conf) const { 555 | char* int_buffer = (char *)malloc(sizeof(char)*kRecordSize); 556 | for (uint32 j = 0; j < kRecordSize - 1; j++) { 557 | int_buffer[j] = (rand() % 26 + 'a'); 558 | } 559 | int_buffer[kRecordSize - 1] = '\0'; 560 | 561 | for (uint64 i = 0; i < nparts*kDBSize; i++) { 562 | if (conf->LookupPartition(IntToString(i)) == conf->relative_node_id()) { 563 | string value(int_buffer); 564 | uint32 master = conf->LookupMaster(IntToString(i)); 565 | storage->PutObject(IntToString(i), new Record(value, master)); 566 | } 567 | 568 | //if (i % 1000000 == 0) { 569 | // LOG(ERROR) <relative_node_id()<< ":*********In InitializeStorage: Finish 100000 records "; 570 | //} 571 | } 572 | } 573 | 574 | -------------------------------------------------------------------------------- /src/applications/microbenchmark.h: -------------------------------------------------------------------------------- 1 | // Author: Kun Ren 2 | // 3 | // 4 | // A microbenchmark application that reads all elements of the read_set, does 5 | // some trivial computation, and writes to all elements of the write_set. 6 | 7 | #ifndef _DB_APPLICATIONS_MICROBENCHMARK_H_ 8 | #define _DB_APPLICATIONS_MICROBENCHMARK_H_ 9 | 10 | #include 11 | #include 12 | 13 | #include "applications/application.h" 14 | #include "machine/cluster_config.h" 15 | #include "machine/connection.h" 16 | 17 | using std::set; 18 | using std::string; 19 | 20 | class Microbenchmark : public Application { 21 | public: 22 | enum TxnType { 23 | INITIALIZE = 0, 24 | MICROTXN_SP = 1, 25 | MICROTXN_MP = 2, 26 | MICROTXN_SRSP = 3, 27 | MICROTXN_SRMP = 4, 28 | MICROTXN_MRSP = 5, 29 | MICROTXN_MRMP = 6, 30 | }; 31 | 32 | Microbenchmark(ClusterConfig* conf, uint32 hotcount) { 33 | nparts = conf->nodes_per_replica(); 34 | hot_records = hotcount; 35 | replica_size = conf->replicas_size(); 36 | config_ = conf; 37 | local_replica_ = config_->local_replica_id(); 38 | } 39 | 40 | Microbenchmark(ClusterConfig* conf, ConnectionMultiplexer* multiplexer, uint32 hotcount) { 41 | nparts = conf->nodes_per_replica(); 42 | hot_records = hotcount; 43 | replica_size = conf->replicas_size(); 44 | config_ = conf; 45 | local_replica_ = config_->local_replica_id(); 46 | 47 | connection_ = multiplexer; 48 | } 49 | 50 | virtual ~Microbenchmark() {} 51 | 52 | virtual TxnProto* NewTxn(int64 txn_id, int txn_type, string args, 53 | ClusterConfig* config = NULL) const; 54 | 55 | virtual int Execute(TxnProto* txn, StorageManager* storage) const; 56 | 57 | TxnProto* MicroTxnSP(int64 txn_id, uint64 part); 58 | TxnProto* MicroTxnMP(int64 txn_id, uint64 part1, uint64 part2); 59 | TxnProto* MicroTxnSRSP(int64 txn_id, uint64 part, uint32 replica); 60 | TxnProto* MicroTxnSRMP(int64 txn_id, uint64 part1, uint64 part2, uint32 replica); 61 | TxnProto* MicroTxnMRSP(int64 txn_id, uint64 part, uint32 replica1, uint32 replica2); 62 | TxnProto* MicroTxnMRMP(int64 txn_id, uint64 part1, uint64 part2, uint32 replica1, uint32 replica2); 63 | 64 | uint32 nparts; 65 | uint32 hot_records; 66 | uint32 replica_size; 67 | 68 | ClusterConfig* config_; 69 | uint32 local_replica_; 70 | ConnectionMultiplexer* connection_; 71 | 72 | static const uint32 kRWSetSize = 10; // MUST BE EVEN 73 | static const uint64 kDBSize = 10000000; 74 | static const uint32 kRecordSize = 100; 75 | 76 | 77 | virtual void InitializeStorage(Storage* storage, ClusterConfig* conf) const; 78 | 79 | private: 80 | void GetRandomKeys(set* keys, uint32 num_keys, uint64 key_start, 81 | uint64 key_limit, uint64 part); 82 | void GetRandomKeysReplica(set* keys, uint32 num_keys, uint64 key_start, 83 | uint64 key_limit, uint64 part, uint32 replica); 84 | Microbenchmark() {} 85 | }; 86 | 87 | #endif // _DB_APPLICATIONS_MICROBENCHMARK_H_ 88 | -------------------------------------------------------------------------------- /src/applications/tpcc.h: -------------------------------------------------------------------------------- 1 | // Author: Kun Ren 2 | // 3 | // 4 | // A simulated TPC-C application 5 | 6 | #ifndef _DB_APPLICATIONS_TPCC_H_ 7 | #define _DB_APPLICATIONS_TPCC_H_ 8 | 9 | #include 10 | #include 11 | 12 | #include "applications/application.h" 13 | #include "machine/cluster_config.h" 14 | #include "machine/connection.h" 15 | 16 | using std::set; 17 | using std::string; 18 | 19 | #define DISTRICTS_PER_WAREHOUSE 10 20 | #define CUSTOMERS_PER_DISTRICT 3000 21 | #define NUMBER_OF_ITEMS 100000 22 | 23 | class Tpcc : public Application { 24 | public: 25 | enum TxnType { 26 | INITIALIZE = 0, 27 | TPCCTXN_SP = 1, 28 | TPCCTXN_MP = 2, 29 | TPCCTXN_SRSP = 3, 30 | TPCCTXN_SRMP = 4, 31 | TPCCTXN_MRSP = 5, 32 | TPCCTXN_MRMP = 6, 33 | }; 34 | 35 | Tpcc(ClusterConfig* conf, uint32 warehouses) { 36 | nparts = conf->nodes_per_replica(); 37 | warehouses_per_node = warehouses; 38 | replica_size = conf->replicas_size(); 39 | config_ = conf; 40 | local_replica_ = config_->local_replica_id(); 41 | 42 | // For tpcc 43 | warehouse_end = warehouses_per_node * nparts; 44 | district_end = warehouse_end + nparts * (warehouses_per_node * DISTRICTS_PER_WAREHOUSE); 45 | customer_end = district_end + nparts * (warehouses_per_node * DISTRICTS_PER_WAREHOUSE * CUSTOMERS_PER_DISTRICT); 46 | item_end = customer_end + nparts * (warehouses_per_node * NUMBER_OF_ITEMS); 47 | 48 | kDBSize = item_end; 49 | } 50 | 51 | Tpcc(ClusterConfig* conf, ConnectionMultiplexer* multiplexer, uint32 warehouses) { 52 | nparts = conf->nodes_per_replica(); 53 | warehouses_per_node = warehouses; 54 | replica_size = conf->replicas_size(); 55 | config_ = conf; 56 | local_replica_ = config_->local_replica_id(); 57 | 58 | connection_ = multiplexer; 59 | 60 | // For tpcc 61 | warehouse_end = warehouses_per_node * nparts; 62 | district_end = warehouse_end + nparts * (warehouses_per_node * DISTRICTS_PER_WAREHOUSE); 63 | customer_end = district_end + nparts * (warehouses_per_node * DISTRICTS_PER_WAREHOUSE * CUSTOMERS_PER_DISTRICT); 64 | item_end = customer_end + nparts * (warehouses_per_node * NUMBER_OF_ITEMS); 65 | 66 | kDBSize = item_end; 67 | } 68 | 69 | virtual ~Tpcc() {} 70 | 71 | virtual TxnProto* NewTxn(int64 txn_id, int txn_type, string args, 72 | ClusterConfig* config = NULL) const; 73 | 74 | virtual int Execute(TxnProto* txn, StorageManager* storage) const; 75 | 76 | TxnProto* TpccTxnSP(int64 txn_id, uint64 part); 77 | TxnProto* TpccTxnMP(int64 txn_id, uint64 part1, uint64 part2); 78 | TxnProto* TpccTxnSRSP(int64 txn_id, uint64 part, uint32 replica); 79 | TxnProto* TpccTxnSRMP(int64 txn_id, uint64 part1, uint64 part2, uint32 replica); 80 | TxnProto* TpccTxnMRSP(int64 txn_id, uint64 part, uint32 replica1, uint32 replica2); 81 | TxnProto* TpccTxnMRMP(int64 txn_id, uint64 part1, uint64 part2, uint32 replica1, uint32 replica2); 82 | 83 | uint32 nparts; 84 | uint32 warehouses_per_node; 85 | uint32 replica_size; 86 | 87 | // For tpcc 88 | uint64 warehouse_end; 89 | uint64 district_end; 90 | uint64 customer_end; 91 | uint64 item_end; 92 | 93 | ClusterConfig* config_; 94 | uint32 local_replica_; 95 | ConnectionMultiplexer* connection_; 96 | 97 | uint64 kDBSize; 98 | static const uint32 kRecordSize = 100; 99 | 100 | 101 | virtual void InitializeStorage(Storage* storage, ClusterConfig* conf) const; 102 | 103 | private: 104 | void GetRandomKeys(set* keys, uint32 num_keys, uint64 key_start, 105 | uint64 key_limit, uint64 part); 106 | void GetRandomKeysReplica(set* keys, uint32 num_keys, uint64 key_start, 107 | uint64 key_limit, uint64 part, uint32 replica); 108 | Tpcc() {} 109 | }; 110 | 111 | #endif // _DB_APPLICATIONS_MICROBENCHMARK_H_ 112 | -------------------------------------------------------------------------------- /src/backend/Makefile.inc: -------------------------------------------------------------------------------- 1 | # Create template specific variables 2 | UC_DIR := BACKEND 3 | LC_DIR := backend 4 | 5 | SRCS := backend/simple_storage.cc \ 6 | backend/storage_manager.cc 7 | 8 | EXES := 9 | DEPS := $(PROTO_OBJS) $(COMMON_OBJS) 10 | 11 | # Link the template to avoid redundancy 12 | include $(MAKEFILE_TEMPLATE) 13 | 14 | -------------------------------------------------------------------------------- /src/backend/simple_storage.cc: -------------------------------------------------------------------------------- 1 | // Author: Kun Ren 2 | // 3 | // 4 | // A simple implementation of the storage interface using an stl map. 5 | 6 | #include "backend/simple_storage.h" 7 | 8 | Record* SimpleStorage::ReadObject(const Key& key) { 9 | CHECK(objects_.count(key) != 0); 10 | return objects_[key]; 11 | } 12 | 13 | bool SimpleStorage::PutObject(const Key& key, Record* record) { 14 | Lock l(&mutex_); 15 | objects_[key] = record; 16 | return true; 17 | } 18 | 19 | bool SimpleStorage::DeleteObject(const Key& key) { 20 | Lock l(&mutex_); 21 | objects_.erase(key); 22 | return true; 23 | } 24 | 25 | pair SimpleStorage::GetMasterCounter(const Key& key) { 26 | CHECK(objects_.count(key) != 0); 27 | Record* record = objects_[key]; 28 | return make_pair(record->master, record->counter); 29 | } 30 | -------------------------------------------------------------------------------- /src/backend/simple_storage.h: -------------------------------------------------------------------------------- 1 | // Author: Kun Ren 2 | // 3 | // 4 | // A simple implementation of the storage interface using an stl map. 5 | 6 | #ifndef _DB_BACKEND_SIMPLE_STORAGE_H_ 7 | #define _DB_BACKEND_SIMPLE_STORAGE_H_ 8 | 9 | #include 10 | #include "common/utils.h" 11 | #include "common/mutex.h" 12 | #include "backend/storage.h" 13 | #include "common/types.h" 14 | #include 15 | 16 | using std::tr1::unordered_map; 17 | 18 | class SimpleStorage : public Storage { 19 | public: 20 | virtual ~SimpleStorage() {} 21 | 22 | virtual Record* ReadObject(const Key& key); 23 | virtual bool PutObject(const Key& key, Record* record); 24 | virtual bool DeleteObject(const Key& key); 25 | virtual pair GetMasterCounter(const Key& key); 26 | 27 | private: 28 | unordered_map objects_; 29 | Mutex mutex_; 30 | 31 | }; 32 | #endif // _DB_BACKEND_SIMPLE_STORAGE_H_ 33 | 34 | -------------------------------------------------------------------------------- /src/backend/storage.h: -------------------------------------------------------------------------------- 1 | // Author: Kun Ren 2 | // 3 | // The Storage class provides an interface for writing and accessing data 4 | // objects stored by the system. 5 | 6 | #ifndef _DB_BACKEND_STORAGE_H_ 7 | #define _DB_BACKEND_STORAGE_H_ 8 | 9 | #include 10 | #include 11 | #include "common/types.h" 12 | 13 | #define REPLICA_SIZE 3 14 | #define LAST_N_TOUCH 10000 15 | #define ACCESS_PATTERN_THRESHOLD 0.90 16 | 17 | using std::vector; 18 | using std::pair; 19 | using std::make_pair; 20 | 21 | struct Record { 22 | Record(Value v, uint32 m) : value(v), master(m), counter(0) { 23 | for (uint32 i = 0; i < REPLICA_SIZE; i++) { 24 | access_pattern[i] = 0; 25 | } 26 | remastering = false; 27 | access_cnt = 0; 28 | } 29 | 30 | Record(Value v, uint32 m, uint64 c) : value(v), master(m), counter(c) { 31 | for (uint32 i = 0; i < REPLICA_SIZE; i++) { 32 | access_pattern[i] = 0; 33 | } 34 | remastering = false; 35 | access_cnt = 0; 36 | } 37 | 38 | // The actual value 39 | Value value; 40 | uint32 master; 41 | uint64 counter; 42 | 43 | // access pattern 44 | uint32 access_pattern[REPLICA_SIZE]; 45 | bool remastering; 46 | uint32 access_cnt; 47 | 48 | }; 49 | 50 | 51 | class Storage { 52 | public: 53 | virtual ~Storage() {} 54 | 55 | // If the object specified by 'key' exists, copies the object into '*result' 56 | // and returns true. If the object does not exist, false is returned. 57 | virtual Record* ReadObject(const Key& key) = 0; 58 | 59 | // Sets the object specified by 'key' equal to 'value'. Any previous version 60 | // of the object is replaced. Returns true if the write succeeds, or false if 61 | // it fails for any reason. 62 | virtual bool PutObject(const Key& key, Record* record) = 0; 63 | 64 | // Removes the object specified by 'key' if there is one. Returns true if the 65 | // deletion succeeds (or if no object is found with the specified key), or 66 | // false if it fails for any reason. 67 | virtual bool DeleteObject(const Key& key) = 0; 68 | 69 | virtual pair GetMasterCounter(const Key& key) = 0; 70 | 71 | }; 72 | 73 | #endif // _DB_BACKEND_STORAGE_H_ 74 | 75 | -------------------------------------------------------------------------------- /src/backend/storage_manager.cc: -------------------------------------------------------------------------------- 1 | // Author: Kun Ren 2 | // 3 | 4 | #include "backend/storage_manager.h" 5 | #include 6 | 7 | StorageManager::StorageManager(ClusterConfig* config, ConnectionMultiplexer* connection, 8 | Storage* actual_storage, TxnProto* txn, uint32 mode) 9 | : configuration_(config), connection_(connection), 10 | actual_storage_(actual_storage), txn_(txn), mode_(mode), relative_node_id_(config->relative_node_id()){ 11 | local_replica_id_ = config->local_replica_id(); 12 | 13 | // If reads are performed at this node, execute local reads and broadcast 14 | // results to all (other) writers. 15 | set writers; 16 | txn_origin_replica_ = txn->origin_replica(); 17 | local_commit_ = true; 18 | 19 | bool reader = false; 20 | for (int i = 0; i < txn->readers_size(); i++) { 21 | if (txn->readers(i) == relative_node_id_) 22 | reader = true; 23 | } 24 | 25 | if (reader) { 26 | remote_result_message_.set_type(MessageProto::READ_RESULT); 27 | 28 | // Execute local reads. 29 | for (int i = 0; i < txn->read_set_size(); i++) { 30 | KeyEntry key_entry = txn->read_set(i); 31 | const Key& key = key_entry.key(); 32 | uint64 mds = configuration_->LookupPartition(key); 33 | 34 | if (mode_ == 2) { 35 | involved_machines_.insert(make_pair(mds, key_entry.master())); 36 | } 37 | 38 | if (mode_ != 0 && key_entry.master() != txn_origin_replica_) { 39 | continue; 40 | } 41 | 42 | if (mds == relative_node_id_) { 43 | Record* val = actual_storage_->ReadObject(key); 44 | objects_[key] = val; 45 | 46 | // Collect local entries 47 | RemoteResultsEntry* results_entry = local_entries_.add_entries(); 48 | results_entry->set_key(key); 49 | results_entry->set_value(val->value); 50 | results_entry->set_master(val->master); 51 | results_entry->set_counter(val->counter); 52 | 53 | if (val->master != key_entry.master() || val->counter != key_entry.counter()) { 54 | local_commit_ = false; 55 | } 56 | 57 | } 58 | } 59 | 60 | for (int i = 0; i < txn->read_write_set_size(); i++) { 61 | KeyEntry key_entry = txn->read_write_set(i); 62 | const Key& key = key_entry.key(); 63 | uint64 mds = configuration_->LookupPartition(key); 64 | writers.insert(mds); 65 | 66 | if (mode_ == 2) { 67 | involved_machines_.insert(make_pair(mds, key_entry.master())); 68 | } 69 | 70 | uint32 replica_id = key_entry.master(); 71 | if (mode_ != 0 && replica_id != txn_origin_replica_) { 72 | // Record remote writers 73 | remote_replica_writers_.insert(make_pair(mds, replica_id)); 74 | continue; 75 | } else if (mode_ != 0 && mds != relative_node_id_) { 76 | // Record remote writers 77 | remote_replica_writers_.insert(make_pair(mds, txn_origin_replica_)); 78 | continue; 79 | } 80 | 81 | if (mds == relative_node_id_) { 82 | Record* val = actual_storage_->ReadObject(key); 83 | objects_[key] = val; 84 | 85 | // Collect local entries 86 | RemoteResultsEntry* results_entry = local_entries_.add_entries(); 87 | results_entry->set_key(key); 88 | results_entry->set_value(val->value); 89 | results_entry->set_master(val->master); 90 | results_entry->set_counter(val->counter); 91 | 92 | if (val->master != key_entry.master() || val->counter != key_entry.counter()) { 93 | local_commit_ = false; 94 | } 95 | } 96 | } 97 | 98 | if (mode_ == 0) { 99 | // Original CalvinDB: Broadcast local reads to (other) writers. 100 | string local_entries_string; 101 | local_entries_.SerializeToString(&local_entries_string); 102 | remote_result_message_.add_data(local_entries_string); 103 | 104 | remote_result_message_.set_destination_channel(IntToString(txn->txn_id()) + "-" + IntToString(txn_origin_replica_)); 105 | for (set::iterator it = writers.begin(); it != writers.end(); ++it) { 106 | if (*it != relative_node_id_) { 107 | remote_result_message_.set_destination_node(configuration_->LookupMachineID(*it, configuration_->local_replica_id())); 108 | connection_->Send(remote_result_message_); 109 | } 110 | } 111 | } else if (mode_ == 1){ 112 | // Basic request chopping: Broadcast local reads to (other) writers(on the same machine but with different master or on different machine but with same replica). 113 | string local_entries_string; 114 | local_entries_.SerializeToString(&local_entries_string); 115 | remote_result_message_.add_data(local_entries_string); 116 | 117 | for (auto remote_writer : remote_replica_writers_) { 118 | uint64 mds = remote_writer.first; 119 | uint64 replica = remote_writer.second; 120 | string destination_channel = IntToString(txn->txn_id()) + "-" + IntToString(replica); 121 | remote_result_message_.set_destination_channel(destination_channel); 122 | remote_result_message_.set_destination_node(configuration_->LookupMachineID(mds, configuration_->local_replica_id())); 123 | connection_->Send(remote_result_message_); 124 | } 125 | } else { 126 | // (for mp or mr txns) Request chopping with remaster: get the min_involved_machine_ and min_involved_machine_origin_(will generate new txn if aborted) 127 | pair min_machine = make_pair(INT_MAX, INT_MAX); 128 | for (auto machine : involved_machines_) { 129 | if (machine.first < min_machine.first) { 130 | min_machine.first = machine.first; 131 | min_machine.second = machine.second; 132 | } else if (machine.first == min_machine.first && machine.second < min_machine.second) { 133 | min_machine.first = machine.first; 134 | min_machine.second = machine.second; 135 | } 136 | } 137 | 138 | min_involved_machine_ = min_machine.first; 139 | min_involved_machine_origin_ = min_machine.second; 140 | 141 | string local_entries_string; 142 | local_entries_.SerializeToString(&local_entries_string); 143 | remote_result_message_.add_data(local_entries_string); 144 | 145 | for (auto remote_writer : remote_replica_writers_) { 146 | uint64 mds = remote_writer.first; 147 | uint64 replica = remote_writer.second; 148 | string destination_channel = IntToString(txn->txn_id()) + "-" + IntToString(replica); 149 | remote_result_message_.set_destination_channel(destination_channel); 150 | remote_result_message_.set_destination_node(configuration_->LookupMachineID(mds, configuration_->local_replica_id())); 151 | connection_->Send(remote_result_message_); 152 | } 153 | } 154 | 155 | } 156 | 157 | // Note whether this node is a writer. If not, no need to do anything further. 158 | writer = false; 159 | for (int i = 0; i < txn->writers_size(); i++) { 160 | if (txn->writers(i) == relative_node_id_) 161 | writer = true; 162 | } 163 | 164 | // Scheduler is responsible for calling HandleReadResponse. We're done here. 165 | } 166 | /** 167 | bool StorageManager::CheckCommitOrAbort() { 168 | return true; 169 | } **/ 170 | 171 | // For bachground remaster: Return true if it will commit, or return false if it will abort and set the status to ABORTED 172 | bool StorageManager::CheckCommitOrAbort() { 173 | bool decision = true; 174 | 175 | if (txn_->status() == TxnProto::ABORTED_WITHOUT_LOCK) { 176 | // If we already know this txn will be aborted in the locking thread(without acquiring locks), then we will abort it. 177 | decision = false; 178 | //LOG(ERROR) << configuration_->local_node_id()<< ":wrong------------------------"; 179 | } else if (local_commit_ == false) { 180 | // If we know it will be aborted based on the local information 181 | txn_->set_status(TxnProto::ABORTED); 182 | decision = false; 183 | //LOG(ERROR) << configuration_->local_node_id()<< ":wrong------------------------"; 184 | } else if (involved_machines_.size() == 1) { 185 | decision = true; 186 | } else { 187 | // check the all the master/counter 188 | for (int i = 0; i < txn_->read_set_size(); i++) { 189 | KeyEntry key_entry = txn_->read_set(i); 190 | Record* val = objects_[key_entry.key()]; 191 | if (key_entry.master() != val->master || key_entry.counter() != val->counter) { 192 | decision = false; 193 | break; 194 | } 195 | } 196 | 197 | if (decision == true) { 198 | for (int i = 0; i < txn_->read_write_set_size(); i++) { 199 | KeyEntry key_entry = txn_->read_write_set(i); 200 | Record* val = objects_[key_entry.key()]; 201 | if (key_entry.master() != val->master || key_entry.counter() != val->counter) { 202 | decision = false; 203 | break; 204 | } 205 | } 206 | } 207 | 208 | if (decision == false) { 209 | txn_->set_status(TxnProto::ABORTED); 210 | } 211 | 212 | } // end if else 213 | 214 | // If we need to generate new transction for the aborted txn 215 | if (decision == false && local_replica_id_ == txn_origin_replica_ && min_involved_machine_ == relative_node_id_ && min_involved_machine_origin_ == txn_origin_replica_) { 216 | //LOG(ERROR) << configuration_->local_node_id()<< ":Generate new transaction for the aborted txn------------------------:"<txn_id(); 217 | // abort the txn and send it to the related replica. 218 | TxnProto txn; 219 | txn.CopyFrom(*txn_); 220 | txn.clear_involved_replicas(); 221 | 222 | txn.set_txn_id(configuration_->GetGUID()); 223 | txn.set_status(TxnProto::NEW); 224 | 225 | set involved_replicas; 226 | 227 | for (int i = 0; i < txn.read_set_size(); i++) { 228 | KeyEntry key_entry = txn.read_set(i); 229 | Record* val = objects_[key_entry.key()]; 230 | if (key_entry.master() != val->master || key_entry.counter() != val->counter) { 231 | // update to the latest master/counter 232 | txn.mutable_read_set(i)->set_master(val->master); 233 | txn.mutable_read_set(i)->set_counter(val->counter); 234 | } 235 | involved_replicas.insert(val->master); 236 | } 237 | 238 | for (int i = 0; i < txn.read_write_set_size(); i++) { 239 | KeyEntry key_entry = txn.read_write_set(i); 240 | Record* val = objects_[key_entry.key()]; 241 | if (key_entry.master() != val->master || key_entry.counter() != val->counter) { 242 | // update to the latest master/counter 243 | txn.mutable_read_write_set(i)->set_master(val->master); 244 | txn.mutable_read_write_set(i)->set_counter(val->counter); 245 | } 246 | involved_replicas.insert(val->master); 247 | } 248 | 249 | for (auto replica : involved_replicas) { 250 | txn.add_involved_replicas(replica); 251 | } 252 | 253 | MessageProto forward_txn_message_; 254 | forward_txn_message_.set_destination_channel("sequencer_txn_receive_"); 255 | forward_txn_message_.set_type(MessageProto::TXN_FORWORD); 256 | 257 | string txn_string; 258 | txn.SerializeToString(&txn_string); 259 | 260 | if (txn.involved_replicas_size() == 1) { 261 | //LOG(ERROR) << configuration_->local_node_id()<< " :"<txn_id()<<" new id:"<nodes_per_replica() + rand() % configuration_->nodes_per_replica(); 264 | forward_txn_message_.clear_data(); 265 | forward_txn_message_.add_data(txn_string); 266 | forward_txn_message_.set_destination_node(machine_sent); 267 | connection_->Send(forward_txn_message_); 268 | } else { 269 | //LOG(ERROR) << configuration_->local_node_id()<< " :"<nodes_per_replica(); 272 | forward_txn_message_.clear_data(); 273 | forward_txn_message_.add_data(txn_string); 274 | forward_txn_message_.set_destination_node(machine_sent); 275 | connection_->Send(forward_txn_message_); 276 | } 277 | } // end if 278 | 279 | return decision; 280 | } 281 | 282 | // Handle remote results 283 | void StorageManager::HandleReadResult(const MessageProto& message) { 284 | CHECK(message.type() == MessageProto::READ_RESULT); 285 | 286 | RemoteResultsEntries remote_entries; 287 | remote_entries.ParseFromString(message.data(0)); 288 | 289 | for (uint32 i = 0; i < (uint32)remote_entries.entries_size(); i++) { 290 | RemoteResultsEntry key_entry = remote_entries.entries(i); 291 | Record* val = new Record(key_entry.value(), key_entry.master(), key_entry.counter()); 292 | objects_[key_entry.key()] = val; 293 | remote_reads_.push_back(val); 294 | } 295 | } 296 | 297 | bool StorageManager::ReadyToExecute() { 298 | //LOG(ERROR) <local_node_id()<< ":^^^^^^^^^ In StorageManager: bojects size is: "<(objects_.size()); 299 | return static_cast(objects_.size()) == txn_->read_set_size() + txn_->read_write_set_size(); 300 | } 301 | 302 | StorageManager::~StorageManager() { 303 | for (vector::iterator it = remote_reads_.begin(); 304 | it != remote_reads_.end(); ++it) { 305 | delete *it; 306 | } 307 | } 308 | 309 | Record* StorageManager::ReadObject(const Key& key) { 310 | return objects_[key]; 311 | } 312 | 313 | bool StorageManager::PutObject(const Key& key, Record* value) { 314 | // Write object to storage if applicable. 315 | if (configuration_->LookupPartition(key) == relative_node_id_) { 316 | return actual_storage_->PutObject(key, value); 317 | } else { 318 | return true; // Not this node's problem. 319 | } 320 | } 321 | 322 | bool StorageManager::DeleteObject(const Key& key) { 323 | // Delete object from storage if applicable. 324 | if (configuration_->LookupPartition(key) == relative_node_id_) { 325 | return actual_storage_->DeleteObject(key); 326 | } else { 327 | return true; // Not this node's problem. 328 | } 329 | } 330 | 331 | -------------------------------------------------------------------------------- /src/backend/storage_manager.h: -------------------------------------------------------------------------------- 1 | // Author: Kun Ren 2 | // 3 | // 4 | // A wrapper for a storage layer that can be used by an Application to simplify 5 | // application code by hiding all inter-node communication logic. By using this 6 | // class as the primary interface for applications to interact with storage of 7 | // actual data objects, applications can be written without paying any attention 8 | // to partitioning at all. 9 | // 10 | // StorageManager use: 11 | // - Each transaction execution creates a new StorageManager and deletes it 12 | // upon completion. 13 | // - No ReadObject call takes as an argument any value that depends on the 14 | // result of a previous ReadObject call. 15 | // - In any transaction execution, a call to DoneReading must follow ALL calls 16 | // to ReadObject and must precede BOTH (a) any actual interaction with the 17 | // values 'read' by earlier calls to ReadObject and (b) any calls to 18 | // PutObject or DeleteObject. 19 | 20 | #ifndef _DB_BACKEND_STORAGE_MANAGER_H_ 21 | #define _DB_BACKEND_STORAGE_MANAGER_H_ 22 | 23 | #include 24 | 25 | #include 26 | #include 27 | 28 | #include "common/types.h" 29 | #include "backend/storage.h" 30 | #include "machine/cluster_config.h" 31 | #include "machine/connection.h" 32 | #include "common/utils.h" 33 | #include "proto/txn.pb.h" 34 | #include "proto/message.pb.h" 35 | 36 | using std::vector; 37 | using std::tr1::unordered_map; 38 | using std::set; 39 | using std::pair; 40 | using std::make_pair; 41 | 42 | class ClusterConfig; 43 | class Connection; 44 | class MessageProto; 45 | class Scheduler; 46 | class Storage; 47 | class TxnProto; 48 | 49 | class StorageManager { 50 | public: 51 | StorageManager(ClusterConfig* config, ConnectionMultiplexer* connection, 52 | Storage* actual_storage, TxnProto* txn, uint32 mode); 53 | 54 | ~StorageManager(); 55 | 56 | Record* ReadObject(const Key& key); 57 | bool PutObject(const Key& key, Record* record); 58 | bool DeleteObject(const Key& key); 59 | 60 | void HandleReadResult(const MessageProto& message); 61 | bool ReadyToExecute(); 62 | 63 | bool CheckCommitOrAbort(); 64 | 65 | Storage* GetStorage() { return actual_storage_; } 66 | 67 | uint32 GetMode() {return mode_;} 68 | 69 | // Set by the constructor, indicating whether 'txn' involves any writes at 70 | // this node. 71 | bool writer; 72 | 73 | private: 74 | friend class DeterministicScheduler; 75 | 76 | // Pointer to the configuration object for this node. 77 | ClusterConfig* configuration_; 78 | 79 | // A Connection object that can be used to send and receive messages. 80 | ConnectionMultiplexer* connection_; 81 | 82 | // Storage layer that *actually* stores data objects on this node. 83 | Storage* actual_storage_; 84 | 85 | // Transaction that corresponds to this instance of a StorageManager. 86 | TxnProto* txn_; 87 | 88 | uint32 mode_; 89 | 90 | // Local copy of all data objects read/written by 'txn_', populated at 91 | // StorageManager construction time. 92 | // 93 | unordered_map objects_; 94 | 95 | vector remote_reads_; 96 | 97 | uint64 relative_node_id_; 98 | 99 | uint32 local_replica_id_; 100 | 101 | // For request chopping algorithm: pair 102 | set> remote_replica_writers_; 103 | 104 | // remote results message 105 | MessageProto remote_result_message_; 106 | 107 | // For request chopping with remaster 108 | set> involved_machines_; 109 | uint64 min_involved_machine_; 110 | uint32 min_involved_machine_origin_; 111 | 112 | RemoteResultsEntries local_entries_; 113 | uint32 txn_origin_replica_; 114 | 115 | bool local_commit_; 116 | 117 | 118 | }; 119 | 120 | #endif // _DB_BACKEND_STORAGE_MANAGER_H_ 121 | 122 | -------------------------------------------------------------------------------- /src/common/Makefile.inc: -------------------------------------------------------------------------------- 1 | # Create template specific variables 2 | UC_DIR := COMMON 3 | LC_DIR := common 4 | 5 | SRCS := common/utils.cc 6 | 7 | EXES := 8 | 9 | DEPS := $(PROTO_OBJS) 10 | 11 | # Link the template to avoid redundancy 12 | include $(MAKEFILE_TEMPLATE) 13 | 14 | -------------------------------------------------------------------------------- /src/common/mutex.h: -------------------------------------------------------------------------------- 1 | // Author: Alex Thomson (thomson@cs.yale.edu) 2 | // 3 | // Mutex is currently a thin wrapper around pthread_mutex_t, but this may 4 | // change in the future, so please use this Mutex class rather than 5 | // pthread_mutex_t for critical sections in Calvin. 6 | // 7 | // Standard usage idiom: 8 | // 9 | // Mutex m; // create a new mutex 10 | // 11 | // void DoSomething() { 12 | // Lock l(&m); // creating a 'Lock' object locks the mutex 13 | // 14 | // } // when the Lock object goes out of scope and is 15 | // // deallocated, the mutex is automatically 16 | // // unlocked 17 | // 18 | 19 | #ifndef CALVIN_COMMON_MUTEX_H_ 20 | #define CALVIN_COMMON_MUTEX_H_ 21 | 22 | #include 23 | #include 24 | 25 | class Mutex { 26 | public: 27 | // Mutexes come into the world unlocked. 28 | Mutex() { 29 | pthread_mutex_init(&mutex_, NULL); 30 | } 31 | 32 | private: 33 | friend class Lock; 34 | // Actual pthread mutex wrapped by Mutex class. 35 | pthread_mutex_t mutex_; 36 | 37 | // DISALLOW_COPY_AND_ASSIGN 38 | Mutex(const Mutex&); 39 | Mutex& operator=(const Mutex&); 40 | }; 41 | 42 | class Lock { 43 | public: 44 | explicit Lock(Mutex* mutex) : mutex_(mutex) { 45 | pthread_mutex_lock(&mutex_->mutex_); 46 | } 47 | ~Lock() { 48 | pthread_mutex_unlock(&mutex_->mutex_); 49 | } 50 | 51 | private: 52 | Mutex* mutex_; 53 | 54 | // DISALLOW_DEFAULT_CONSTRUCTOR 55 | Lock(); 56 | 57 | // DISALLOW_COPY_AND_ASSIGN 58 | Lock(const Lock&); 59 | Lock& operator=(const Lock&); 60 | }; 61 | 62 | class MutexRW { 63 | public: 64 | // Mutexes come into the world unlocked. 65 | MutexRW() { 66 | pthread_rwlock_init(&mutex_, NULL); 67 | } 68 | 69 | private: 70 | friend class ReadLock; 71 | friend class WriteLock; 72 | // Actual pthread rwlock wrapped by MutexRW class. 73 | pthread_rwlock_t mutex_; 74 | 75 | // DISALLOW_COPY_AND_ASSIGN 76 | MutexRW(const MutexRW&); 77 | MutexRW& operator=(const MutexRW&); 78 | }; 79 | 80 | class ReadLock { 81 | public: 82 | explicit ReadLock(MutexRW* mutex) : mutex_(mutex) { 83 | pthread_rwlock_rdlock(&mutex_->mutex_); 84 | } 85 | ~ReadLock() { 86 | pthread_rwlock_unlock(&mutex_->mutex_); 87 | } 88 | 89 | private: 90 | MutexRW* mutex_; 91 | 92 | // DISALLOW_DEFAULT_CONSTRUCTOR 93 | ReadLock(); 94 | 95 | // DISALLOW_COPY_AND_ASSIGN 96 | ReadLock(const ReadLock&); 97 | ReadLock& operator=(const ReadLock&); 98 | }; 99 | 100 | class WriteLock { 101 | public: 102 | explicit WriteLock(MutexRW* mutex) : mutex_(mutex) { 103 | pthread_rwlock_wrlock(&mutex_->mutex_); 104 | } 105 | ~WriteLock() { 106 | pthread_rwlock_unlock(&mutex_->mutex_); 107 | } 108 | 109 | private: 110 | MutexRW* mutex_; 111 | 112 | // DISALLOW_DEFAULT_CONSTRUCTOR 113 | WriteLock(); 114 | 115 | // DISALLOW_COPY_AND_ASSIGN 116 | WriteLock(const WriteLock&); 117 | WriteLock& operator=(const WriteLock&); 118 | }; 119 | 120 | #endif // CALVIN_COMMON_MUTEX_H_ 121 | 122 | -------------------------------------------------------------------------------- /src/common/types.h: -------------------------------------------------------------------------------- 1 | // Author: Kun Ren 2 | // 3 | // Just some shorthand typedefs for commonly used types. 4 | 5 | #ifndef CALVIN_COMMON_TYPES_H_ 6 | #define CALVIN_COMMON_TYPES_H_ 7 | 8 | #include 9 | #include // NOLINT 10 | #include 11 | 12 | // Slice and string are common enough that they're worth including here. 13 | using std::string; 14 | 15 | // Abbreviated signed int types. 16 | typedef int8_t int8; 17 | typedef int16_t int16; 18 | typedef int32_t int32; 19 | typedef int64_t int64; 20 | 21 | // Abbreviated unsigned int types. 22 | typedef uint8_t uint8; 23 | typedef uint16_t uint16; 24 | typedef uint32_t uint32; 25 | typedef uint64_t uint64; 26 | 27 | // 'bytes' is an arbitrary sequence of bytes, represented as a string. 28 | typedef string bytes; 29 | 30 | // Key type for database objects. 31 | // Note: if this changes from bytes, the types need to be updated for the 32 | // following fields in .proto files: 33 | // proto/txn.proto: 34 | // TxnProto::'read_set' 35 | // TxnProto::'write_set' 36 | typedef bytes Key; 37 | 38 | // Value type for database objects. 39 | typedef bytes Value; 40 | 41 | #endif // CALVIN_COMMON_TYPES_H_ 42 | 43 | -------------------------------------------------------------------------------- /src/common/utils.cc: -------------------------------------------------------------------------------- 1 | // Author: Alexander Thomson 2 | // Author: Thaddeus Diamond 3 | 4 | #include "common/utils.h" 5 | 6 | #include 7 | 8 | template string TypeName() { 9 | return "?"; 10 | } 11 | ADD_TYPE_NAME(int32); 12 | ADD_TYPE_NAME(string); 13 | 14 | vector SplitString(const string& input, char delimiter) { 15 | string current; 16 | vector result; 17 | for (uint32 i = 0; i < input.size(); i++) { 18 | if (input[i] == delimiter) { 19 | // reached delimiter 20 | result.push_back(current); 21 | current.clear(); 22 | } else { 23 | current.push_back(input[i]); 24 | } 25 | } 26 | result.push_back(current); 27 | return result; 28 | } 29 | 30 | double GetTime() { 31 | struct timeval tv; 32 | gettimeofday(&tv, NULL); 33 | return tv.tv_sec + tv.tv_usec/1e6; 34 | } 35 | 36 | uint32 FNVHash(const string& key) { 37 | uint32 hash = 2166136261; // FNV Hash offset 38 | for (uint32 i = 0; i < key.size(); i++) { 39 | hash = (hash * 1099511628211) ^ key[i]; // H(x) = H(x-1) * FNV' XOR x 40 | } 41 | return hash; 42 | } 43 | 44 | uint32 FNVModHash(const string& key) { 45 | uint32 hash = 2166136261; // FNV Hash offset 46 | for (uint32 i = 0; i < key.size(); i++) { 47 | hash = (hash ^ key[i]) * 1099511628211; // H(x) = H(x-1) * FNV' XOR x 48 | } 49 | return hash; 50 | } 51 | 52 | void Spin(double duration) { 53 | usleep(1000000 * duration); 54 | } 55 | 56 | void SpinUntil(double time) { 57 | if (time > GetTime()) { 58 | usleep(1000000 * (time - GetTime())); 59 | } 60 | } 61 | 62 | string RandomString(int length) { 63 | string s; 64 | for (int i = 0; i < length; i++) { 65 | s += rand() % 26 + 'A'; 66 | } 67 | return s; 68 | } 69 | 70 | string RandomBytes(int length) { 71 | string s; 72 | for (int i = 0; i < length; i++) { 73 | s += rand() % 256; 74 | } 75 | return s; 76 | } 77 | 78 | string RandomBytesNoZeros(int length) { 79 | string s; 80 | for (int i = 0; i < length; i++) { 81 | s += 1 + rand() % 255; 82 | } 83 | return s; 84 | } 85 | 86 | #define RANF() ((double)rand()/(1.0+(double)RAND_MAX)) 87 | double RandomGaussian(double s) { 88 | double x1, x2, w, z; 89 | do { 90 | x1 = 2.0 * RANF() - 1.0; 91 | x2 = 2.0 * RANF() - 1.0; 92 | w = x1 * x1 + x2 * x2; 93 | } while ( w >= 1.0 ); 94 | return s * x1 * sqrt( (-2.0 * log( w ) ) / w ); 95 | z = 0.5 + s * x1 * sqrt( (-2.0 * log( w ) ) / w ); 96 | if (z < 0 || z >= 1) return RANF(); 97 | else return z; 98 | } 99 | 100 | string Int32ToString(int32 n) { 101 | char s[64]; 102 | snprintf(s, sizeof(s), "%d", n); 103 | return string(s); 104 | } 105 | 106 | string Int64ToString(int64 n) { 107 | char s[64]; 108 | snprintf(s, sizeof(s), "%ld", n); 109 | return string(s); 110 | } 111 | 112 | string UInt32ToString(uint32 n) { 113 | char s[64]; 114 | snprintf(s, sizeof(s), "%u", n); 115 | return string(s); 116 | } 117 | 118 | string UInt64ToString(uint64 n) { 119 | char s[64]; 120 | snprintf(s, sizeof(s), "%lu", n); 121 | return string(s); 122 | } 123 | 124 | // Returns a human-readable string representation of an int. 125 | string IntToString(int n) { 126 | char s[64]; 127 | snprintf(s, sizeof(s), "%d", n); 128 | return string(s); 129 | } 130 | 131 | // Converts a human-readable numeric string to an int. 132 | int StringToInt(const string& s) { 133 | return atoi(s.c_str()); 134 | } 135 | 136 | string DoubleToString(double n) { 137 | char s[64]; 138 | snprintf(s, sizeof(s), "%f", n); 139 | return string(s); 140 | } 141 | 142 | 143 | template 144 | void SpinUntilNE(T& t, const T& v) { 145 | while (t == v) { 146 | usleep(10); 147 | } 148 | } 149 | 150 | class MessageBuffer; 151 | template<> 152 | void SpinUntilNE( 153 | MessageBuffer*& t, 154 | MessageBuffer* const& v) { 155 | while (t == v) { 156 | usleep(10); 157 | } 158 | } 159 | 160 | 161 | 162 | -------------------------------------------------------------------------------- /src/common/utils.h: -------------------------------------------------------------------------------- 1 | // Author Alexander Thomson 2 | // Author Thaddeus Diamond 3 | // 4 | // Some miscellaneous commonly-used utility functions. 5 | // 6 | // TODO(alex): Organize these into reasonable categories, etc. 7 | // TODO(alex): MORE/BETTER UNIT TESTING! 8 | 9 | #ifndef CALVIN_COMMON_UTILS_H_ 10 | #define CALVIN_COMMON_UTILS_H_ 11 | 12 | #include 13 | #include 14 | #include 15 | #include 16 | #include 17 | #include 18 | #include 19 | #include 20 | 21 | #include "common/types.h" 22 | #include "common/mutex.h" 23 | 24 | using std::string; 25 | using std::vector; 26 | using std::tr1::unordered_map; 27 | 28 | template string TypeName(); 29 | #define ADD_TYPE_NAME(T) \ 30 | template<> string TypeName() { return #T; } 31 | 32 | // Splits a string on 'delimiter'. 33 | vector SplitString(const string& input, char delimiter); 34 | 35 | // Returns the number of seconds since midnight according to local system time, 36 | // to the nearest microsecond. 37 | double GetTime(); 38 | 39 | // The FNV-1 and FNV-1a hashes as described by Fowler-Noll-Vo. 40 | uint32 FNVHash(const string& key); 41 | uint32 FNVModHash(const string& key); 42 | 43 | // Busy-wait (or yield) for 'duration' seconds. 44 | void Spin(double duration); 45 | 46 | // Busy-wait (or yield) until GetTime() >= 'time'. 47 | void SpinUntil(double time); 48 | 49 | // Produces a random alphabetic string of 'length' characters. 50 | string RandomString(int length); 51 | 52 | // Random byte sequence. Any byte may appear. 53 | string RandomBytes(int length); 54 | 55 | // Random byte sequence. Any byte except '\0' may appear. 56 | string RandomBytesNoZeros(int length); 57 | 58 | double RandomGaussian(double s); 59 | 60 | // Returns human-readable numeric string representation of an (u)int{32,64}. 61 | string Int32ToString(int32 n); 62 | string Int64ToString(int64 n); 63 | string UInt32ToString(uint32 n); 64 | string UInt64ToString(uint64 n); 65 | string DoubleToString(double n); 66 | 67 | string IntToString(int n); 68 | 69 | // Converts a human-readable numeric string to an (u)int{32,64}. Dies on bad 70 | // inputs. 71 | int StringToInt(const string& s); 72 | 73 | static inline void DeleteString(void* data, void* hint) { 74 | delete reinterpret_cast(hint); 75 | } 76 | 77 | // Used for fooling the g++ optimizer. 78 | template 79 | void SpinUntilNE(T& t, const T& v); 80 | 81 | 82 | template 83 | class AtomicQueue { 84 | public: 85 | AtomicQueue() { 86 | queue_.resize(256); 87 | size_ = 256; 88 | front_ = 0; 89 | back_ = 0; 90 | } 91 | 92 | // Returns the number of elements currently in the queue. 93 | inline size_t Size() { 94 | Lock l(&size_mutex_); 95 | return (back_ + size_ - front_) % size_; 96 | } 97 | 98 | // Returns true iff the queue is empty. 99 | inline bool Empty() { 100 | return front_ == back_; 101 | } 102 | 103 | // Atomically pushes 'item' onto the queue. 104 | inline void Push(const T& item) { 105 | Lock l(&back_mutex_); 106 | // Check if the buffer has filled up. Acquire all locks and resize if so. 107 | if (front_ == (back_+1) % size_) { 108 | Lock m(&front_mutex_); 109 | Lock n(&size_mutex_); 110 | uint32 count = (back_ + size_ - front_) % size_; 111 | queue_.resize(size_ * 2); 112 | for (uint32 i = 0; i < count; i++) { 113 | queue_[size_+i] = queue_[(front_ + i) % size_]; 114 | } 115 | front_ = size_; 116 | back_ = size_ + count; 117 | size_ *= 2; 118 | } 119 | // Push item to back of queue. 120 | queue_[back_] = item; 121 | back_ = (back_ + 1) % size_; 122 | } 123 | 124 | // If the queue is non-empty, (atomically) sets '*result' equal to the front 125 | // element, pops the front element from the queue, and returns true, 126 | // otherwise returns false. 127 | inline bool Pop(T* result) { 128 | Lock l(&front_mutex_); 129 | if (front_ != back_) { 130 | *result = queue_[front_]; 131 | front_ = (front_ + 1) % size_; 132 | return true; 133 | } 134 | return false; 135 | } 136 | 137 | // Sets *result equal to the front element and returns true, unless the 138 | // queue is empty, in which case does nothing and returns false. 139 | inline bool Front(T* result) { 140 | Lock l(&front_mutex_); 141 | if (front_ != back_) { 142 | *result = queue_[front_]; 143 | return true; 144 | } 145 | return false; 146 | } 147 | 148 | private: 149 | vector queue_; // Circular buffer containing elements. 150 | uint32 size_; // Allocated size of queue_, not number of elements. 151 | uint32 front_; // Offset of first (oldest) element. 152 | uint32 back_; // First offset following all elements. 153 | 154 | // Mutexes for synchronization. 155 | Mutex front_mutex_; 156 | Mutex back_mutex_; 157 | Mutex size_mutex_; 158 | 159 | // DISALLOW_COPY_AND_ASSIGN 160 | AtomicQueue(const AtomicQueue&); 161 | AtomicQueue& operator=(const AtomicQueue&); 162 | }; 163 | 164 | template 165 | class AtomicMap { 166 | public: 167 | AtomicMap() {} 168 | ~AtomicMap() {} 169 | 170 | inline V Lookup(const K& k) { 171 | ReadLock l(&mutex_); 172 | CHECK(map_.count(k) > 0); 173 | return map_[k]; 174 | } 175 | 176 | inline void Put(const K& k, const V& v) { 177 | WriteLock l(&mutex_); 178 | map_[k] = v; 179 | } 180 | 181 | inline void EraseAndPut(const K& k, const V& v) { 182 | WriteLock l(&mutex_); 183 | map_.erase(k); 184 | map_[k] = v; 185 | } 186 | 187 | inline void Erase(const K& k) { 188 | WriteLock l(&mutex_); 189 | map_.erase(k); 190 | } 191 | 192 | // Puts (k, v) if there is no record for k. Returns the value of v that is 193 | // associated with k afterwards (either the inserted value or the one that 194 | // was there already). 195 | inline V PutNoClobber(const K& k, const V& v) { 196 | WriteLock l(&mutex_); 197 | if (map_.count(k) != 0) { 198 | return map_[k]; 199 | } else { 200 | map_[k] = v; 201 | return v; 202 | } 203 | } 204 | 205 | inline uint32 Size() { 206 | ReadLock l(&mutex_); 207 | return map_.size(); 208 | } 209 | 210 | inline uint32 Count(const K& k) { 211 | ReadLock l(&mutex_); 212 | return map_.count(k); 213 | } 214 | 215 | inline void Destroy() { 216 | WriteLock l(&mutex_); 217 | for (auto it = map_.begin(); it != map_.end(); ++it) { 218 | if (it->second == NULL) { 219 | delete it->second; 220 | } 221 | Erase(it->first); 222 | } 223 | } 224 | 225 | private: 226 | unordered_map map_; 227 | MutexRW mutex_; 228 | 229 | // DISALLOW_COPY_AND_ASSIGN 230 | AtomicMap(const AtomicMap&); 231 | AtomicMap& operator=(const AtomicMap&); 232 | }; 233 | 234 | #endif // CALVIN_COMMON_UTILS_H_ 235 | 236 | -------------------------------------------------------------------------------- /src/log/Makefile.inc: -------------------------------------------------------------------------------- 1 | # Create template specific variables 2 | UC_DIR := LOG 3 | LC_DIR := log 4 | 5 | SRCS := log/paxos.cc \ 6 | log/local_mem_log.cc \ 7 | log/local_paxos.cc 8 | 9 | EXES := 10 | 11 | DEPS := $(PROTO_OBJS) $(COMMON_OBJS) 12 | 13 | # Link the template to avoid redundancy 14 | include $(MAKEFILE_TEMPLATE) 15 | 16 | -------------------------------------------------------------------------------- /src/log/local_mem_log.cc: -------------------------------------------------------------------------------- 1 | // Author: Kun Ren 2 | // 3 | 4 | #include "log/local_mem_log.h" 5 | 6 | #include 7 | #include 8 | 9 | 10 | using std::string; 11 | 12 | class LocalMemLogReader : public Log::Reader { 13 | public: 14 | virtual ~LocalMemLogReader() {} 15 | virtual bool Valid(); 16 | virtual void Reset(); 17 | virtual bool Next(); 18 | virtual bool Seek(uint64 target); 19 | virtual uint64 Version(); 20 | virtual string Entry(); 21 | 22 | private: 23 | friend class LocalMemLog; 24 | 25 | // Constructor called by LocalMemLog::GetReader(); 26 | explicit LocalMemLogReader(LocalMemLog* log); 27 | 28 | // Log whose entries this reader exposes. 29 | LocalMemLog* log_; 30 | 31 | // False iff positioned before the first entry. 32 | bool started_; 33 | 34 | // Vector offset of log entry at which reader is currently positioned. 35 | uint64 offset_; 36 | 37 | // Copy of current entry. 38 | LocalMemLog::Entry entry_; 39 | 40 | // DISALLOW_COPY_AND_ASSIGN 41 | LocalMemLogReader(const LocalMemLogReader&); // NOLINT 42 | LocalMemLogReader& operator=(const LocalMemLogReader&); 43 | }; 44 | 45 | ////////////////////////// LocalMemLog /////////////////////////////// 46 | 47 | LocalMemLog::LocalMemLog() : max_version_(0), size_(0), allocated_(1024*1024) { 48 | entries_ = reinterpret_cast(malloc(allocated_ * sizeof(Entry))); 49 | } 50 | 51 | LocalMemLog::~LocalMemLog() { 52 | free(entries_); 53 | } 54 | 55 | void LocalMemLog::Append(uint64 version, const string& entry) { 56 | CHECK_GE(version, max_version_); 57 | 58 | // Resize if necessary. 59 | if (size_.load() >= allocated_) { 60 | CHECK_EQ(allocated_, size_.load()); 61 | WriteLock l(&mutex_); 62 | allocated_ *= 2; 63 | entries_ = reinterpret_cast( 64 | realloc(entries_, allocated_ * sizeof(Entry))); 65 | } 66 | 67 | entries_[size_.load()] = Entry(version, entry); 68 | size_++; 69 | max_version_ = version; 70 | } 71 | 72 | uint64 LocalMemLog::LastVersion() { 73 | return max_version_; 74 | } 75 | 76 | typename Log::Reader* LocalMemLog::GetReader() { 77 | return new LocalMemLogReader(this); 78 | } 79 | 80 | ////////////////////////// LocalMemLogReader /////////////////////////////// 81 | 82 | LocalMemLogReader::LocalMemLogReader(LocalMemLog* log) 83 | : log_(log), started_(false), offset_(0) { 84 | } 85 | 86 | bool LocalMemLogReader::Valid() { 87 | return started_; 88 | } 89 | 90 | void LocalMemLogReader::Reset() { 91 | started_ = false; 92 | } 93 | 94 | bool LocalMemLogReader::Next() { 95 | if (log_->size_.load() == 0 || 96 | (started_ && offset_ == log_->size_.load() - 1)) { 97 | return false; 98 | } 99 | if (!started_) { 100 | started_ = true; 101 | offset_ = 0; 102 | } else { 103 | offset_++; 104 | } 105 | CHECK(offset_ < log_->size_.load()); 106 | 107 | // Prevent resizing of array while reading entry. 108 | ReadLock l(&log_->mutex_); 109 | entry_ = log_->entries_[offset_]; 110 | 111 | return true; 112 | } 113 | 114 | bool LocalMemLogReader::Seek(uint64 target) { 115 | CHECK(target > 0) << "seeking to invalid version 0"; 116 | if (target > log_->LastVersion()) { 117 | return false; 118 | } 119 | 120 | // Use snapshot of size as of when seek was called. 121 | uint64 size = log_->size_.load(); 122 | 123 | // Seek WILL succeed. 124 | started_ = true; 125 | 126 | // Prevent array resizing. 127 | ReadLock l(&log_->mutex_); 128 | 129 | // Okay, let's see if I remember how to implement binary search. =) 130 | uint64 min = 0; 131 | uint64 max = size - 1; 132 | LocalMemLog::Entry* e = log_->entries_; // For brevity. 133 | 134 | // Check min and max so we can use strict inequalities in invariant. 135 | if (e[min].version == target) { 136 | offset_ = min; 137 | entry_ = e[offset_]; 138 | return true; 139 | } else if (e[max].version == target) { 140 | offset_ = max; 141 | entry_ = e[offset_]; 142 | return true; 143 | } 144 | 145 | // Invariant: e[min].version < target < e[max].version 146 | while (max > min + 1) { 147 | uint64 mid = (min + max) / 2; 148 | if (e[mid].version == target) { 149 | offset_ = mid; 150 | entry_ = e[offset_]; 151 | return true; 152 | } else if (target < e[mid].version) { 153 | max = mid; 154 | } else { // target > e[mid].version 155 | min = mid; 156 | } 157 | } 158 | 159 | // Just to be safe.... 160 | CHECK(e[min].version < target); 161 | CHECK(e[max].version > target); 162 | 163 | offset_ = max; 164 | entry_ = e[offset_]; 165 | return true; 166 | } 167 | 168 | uint64 LocalMemLogReader::Version() { 169 | CHECK(Valid()) << "version called on invalid LogReader"; 170 | return entry_.version; 171 | } 172 | 173 | string LocalMemLogReader::Entry() { 174 | CHECK(Valid()) << "entry called on invalid LogReader"; 175 | return *(entry_.entry); 176 | } 177 | 178 | -------------------------------------------------------------------------------- /src/log/local_mem_log.h: -------------------------------------------------------------------------------- 1 | // Author: Kun Ren 2 | // 3 | // Super-simple in-memory implementation of a components log. The log entries 4 | // are in fact specific to a given Log object. This is not in any way a 5 | // distributed log. 6 | 7 | #ifndef CALVIN_LOG_LOCAL_MEM_LOG_H_ 8 | #define CALVIN_LOG_LOCAL_MEM_LOG_H_ 9 | 10 | #include 11 | #include 12 | 13 | #include "log/log.h" 14 | #include "common/mutex.h" 15 | #include "common/types.h" 16 | #include "common/utils.h" 17 | 18 | 19 | using std::atomic; 20 | using std::vector; 21 | 22 | class LocalMemLog : public Log { 23 | public: 24 | // Initially empty log. 25 | LocalMemLog(); 26 | virtual ~LocalMemLog(); 27 | 28 | // Actual log interface. 29 | virtual void Append(uint64 version, const string& entry); 30 | virtual typename Log::Reader* GetReader(); 31 | virtual uint64 LastVersion(); 32 | 33 | private: 34 | friend class LocalMemLogReader; 35 | 36 | // Mutex guarding state. 37 | MutexRW mutex_; 38 | 39 | // Max version that appears in log so far. 40 | uint64 max_version_; 41 | 42 | // Array of entries. 43 | struct Entry { 44 | Entry() : version(0), entry(NULL) {} 45 | Entry(uint64 v, const string e) 46 | : version(v), entry(new string(e)) { 47 | } 48 | 49 | uint64 version; 50 | string* entry; 51 | }; 52 | Entry* entries_; 53 | atomic size_; 54 | uint64 allocated_; 55 | 56 | // DISALLOW_COPY_AND_ASSIGN 57 | LocalMemLog(const LocalMemLog&); 58 | LocalMemLog& operator=(const LocalMemLog&); 59 | }; 60 | 61 | #endif // CALVIN_COMPONENTS_LOG_LOCAL_MEM_LOG_H_ 62 | 63 | -------------------------------------------------------------------------------- /src/log/local_paxos.h: -------------------------------------------------------------------------------- 1 | // Author: Kun Ren 2 | // 3 | #ifndef CALVIN_LOG_LOCALPAXOS_H_ 4 | #define CALVIN_LOG_LOCALPAXOS_H_ 5 | 6 | #include 7 | #include 8 | #include 9 | #include 10 | #include 11 | #include 12 | 13 | #include "proto/scalar.pb.h" 14 | #include "common/mutex.h" 15 | #include "common/types.h" 16 | #include "log/local_mem_log.h" 17 | #include "proto/scalar.pb.h" 18 | #include "machine/connection.h" 19 | #include "proto/txn.pb.h" 20 | 21 | using std::vector; 22 | using std::atomic; 23 | using std::make_pair; 24 | using std::pair; 25 | using std::queue; 26 | using std::set; 27 | 28 | 29 | class LocalPaxos { 30 | public: 31 | LocalPaxos(ClusterConfig* config, ConnectionMultiplexer* connection, uint32 type); 32 | 33 | ~LocalPaxos(); 34 | 35 | void Stop(); 36 | void Append(uint64 blockid); 37 | 38 | void ReceiveMessage(); 39 | 40 | void HandleRemoteBatch(); 41 | private: 42 | 43 | // Functions to start the Multiplexor's main loops, called in new pthreads by 44 | // the Sequencer's constructor. 45 | static void* RunLeaderThread(void *arg); 46 | static void* RunFollowerThread(void *arg); 47 | 48 | static void* RunLeaderThreadStrong(void *arg); 49 | 50 | // Returns true iff leader. 51 | bool IsLeader(); 52 | 53 | // Leader's main loop. 54 | void RunLeader(); 55 | 56 | void RunLeaderStrong(); 57 | 58 | // Followers' main loop. 59 | void RunFollower(); 60 | 61 | // Participant list. 62 | vector participants_; 63 | 64 | // True iff main thread SHOULD run. 65 | bool go_; 66 | 67 | // check if it is strong availability or not 68 | uint32 type_; 69 | 70 | // Current request sequence that will get replicated. 71 | Sequence sequence_; 72 | std::atomic local_count_; 73 | Mutex mutex_; 74 | 75 | Log* local_log_; 76 | Log* global_log_; 77 | ClusterConfig* configuration_; 78 | uint64 this_machine_id_; 79 | 80 | ConnectionMultiplexer* connection_; 81 | 82 | // Separate pthread contexts in which to run the leader or follower thread. 83 | pthread_t leader_thread_; 84 | pthread_t follower_thread_; 85 | 86 | map mr_txn_batches_; 87 | AtomicQueue> sequences_other_replicas_; 88 | 89 | map readers_for_local_log_; 90 | 91 | set new_sequence_todo; 92 | 93 | uint64 local_next_version; 94 | uint64 global_next_version; 95 | 96 | uint64 machines_per_replica_; 97 | uint32 local_replica_; 98 | 99 | bool received_synchronize_ack; 100 | 101 | // for strong availbility 102 | uint64 quorum_; 103 | MessageProto remote_batch_message_; 104 | }; 105 | 106 | #endif // CALVIN_LOG_LOCALPAXOS_H_ 107 | -------------------------------------------------------------------------------- /src/log/log.h: -------------------------------------------------------------------------------- 1 | // Author: Kun Ren 2 | // 3 | 4 | #ifndef CALVIN_LOG_LOG_H_ 5 | #define CALVIN_LOG_LOG_H_ 6 | 7 | #include "common/types.h" 8 | 9 | // Interface for a components Log. 10 | class Log { 11 | public: 12 | virtual ~Log() {} 13 | // Request to append to the log. Currently only synchronous appends are 14 | // supported. Implementations of the Log interface are NOT required to allow 15 | // multiple concurrent Appends (i.e. multiple Appends generally require 16 | // external synchronization), but concurrent calls to GetReader and 17 | // methods of Log::Reader must be allowed from different threads without 18 | // external synchronization with one another or with Appenders. 19 | // 20 | // Requires: version > 0 21 | // Requires: version > any previously appended version 22 | virtual void Append(uint64 version, const string& entry) = 0; 23 | 24 | // Returns the highest version that has been appended to the log so far, or 25 | // 0 if the log is empty. 26 | virtual uint64 LastVersion() = 0; 27 | 28 | // The Log::Reader functions like an iterator over a Log's entries. Newly 29 | // created LogReaders are positioned logically BEFORE the first log entry. 30 | // 31 | // Log::Readers need not be thread safe. Multiple threads may NOT access 32 | // a single Reader concurrently without external synchronization. 33 | // HOWEVER, a Reader must continue to work even as the underlying Log is 34 | // modified. In addition, as entries are appended to the underlying Log, 35 | // all existing Readers of that log must thereafter reflect the new state of 36 | // the underlying Log. 37 | class Reader { 38 | public: 39 | virtual ~Reader() {} 40 | 41 | // Returns true iff the LogReader currently exposes a valid log entry. 42 | virtual bool Valid() = 0; 43 | 44 | // Resets the LogReader to be positioned before the first log entry. 45 | virtual void Reset() = 0; 46 | 47 | // Attempts to advance the LogReader to the next entry. Returns true on 48 | // success. If no entry exists following the current one, or if the reader 49 | // is unable to advance for some other reason, the LogReader stays 50 | // positioned where it is, and false is returned. 51 | virtual bool Next() = 0; 52 | 53 | // Attempts to position the LogReader at the first entry with whose version 54 | // is greater than or equal to 'target'. Returns true on success. 55 | // If no such exists exists, the LogReader stays positioned where it is, 56 | // and false is returned. 57 | virtual bool Seek(uint64 target) = 0; 58 | 59 | // Return the version of the current entry. 60 | // 61 | // Requires: Valid() 62 | virtual uint64 Version() = 0; 63 | 64 | // Returns the current entry. 65 | // 66 | // Requires: Valid() 67 | virtual string Entry() = 0; 68 | }; 69 | 70 | // Returns a reader for reading the Log. This is required to be thread-safe: 71 | // multiple threads must be able to call 'GetReader' concurrently without 72 | // external synchronization. 73 | virtual Reader* GetReader() = 0; 74 | }; 75 | 76 | #endif // CALVIN_LOG_LOG_H_ 77 | 78 | -------------------------------------------------------------------------------- /src/log/paxos.cc: -------------------------------------------------------------------------------- 1 | // Author: Kun Ren 2 | // 3 | 4 | #include "log/paxos.h" 5 | 6 | 7 | Paxos::Paxos(Log* log, ClusterConfig* config, ConnectionMultiplexer* connection) 8 | : log_(log), configuration_(config), connection_(connection) { 9 | // Init participants_ 10 | participants_.push_back(0); 11 | go_ = true; 12 | count_ = 0; 13 | 14 | for (uint32 i = 1; i < configuration_->replicas_size(); i++) { 15 | participants_.push_back(i * configuration_->nodes_per_replica()); 16 | } 17 | 18 | this_machine_id_ = configuration_->local_node_id(); 19 | 20 | connection_->NewChannel("paxos_log_"); 21 | 22 | cpu_set_t cpuset; 23 | pthread_attr_t attr_writer; 24 | pthread_attr_init(&attr_writer); 25 | CPU_ZERO(&cpuset); 26 | // CPU_SET(2, &cpuset); 27 | // CPU_SET(6, &cpuset); 28 | CPU_SET(7, &cpuset); 29 | pthread_attr_setaffinity_np(&attr_writer, sizeof(cpu_set_t), &cpuset); 30 | 31 | if (IsLeader()) { 32 | pthread_create(&leader_thread_, &attr_writer, RunLeaderThread, reinterpret_cast(this)); 33 | } else { 34 | pthread_create(&follower_thread_, &attr_writer, RunFollowerThread, reinterpret_cast(this)); 35 | } 36 | } 37 | 38 | Paxos::~Paxos() { 39 | Stop(); 40 | if (IsLeader()) { 41 | pthread_join(leader_thread_, NULL); 42 | } else { 43 | pthread_join(follower_thread_, NULL); 44 | } 45 | } 46 | 47 | void* Paxos::RunLeaderThread(void *arg) { 48 | reinterpret_cast(arg)->RunLeader(); 49 | return NULL; 50 | } 51 | 52 | void* Paxos::RunFollowerThread(void *arg) { 53 | reinterpret_cast(arg)->RunFollower(); 54 | return NULL; 55 | } 56 | 57 | bool Paxos::IsLeader() { 58 | return this_machine_id_ == participants_[0]; 59 | } 60 | 61 | void Paxos::Append(uint64 blockid) { 62 | //LOG(ERROR) << "In paxos log: append a batch: "<(participants_.size()) / 2 + 1; 76 | MessageProto sequence_message; 77 | 78 | uint64 machines_per_replica = configuration_->nodes_per_replica(); 79 | uint32 local_replica = configuration_->local_replica_id(); 80 | 81 | while (go_) { 82 | // Sleep while there are NO requests. 83 | while (count_.load() == 0) { 84 | usleep(20); 85 | } 86 | 87 | // Propose a new sequence. 88 | uint64 version; 89 | string encoded; 90 | { 91 | Lock l(&mutex_); 92 | version = next_version; 93 | next_version ++; 94 | sequence_.SerializeToString(&encoded); 95 | sequence_.Clear(); 96 | count_ = 0; 97 | } 98 | 99 | sequence_message.add_data(encoded); 100 | sequence_message.add_misc_int(version); 101 | sequence_message.set_type(MessageProto::PAXOS_DATA); 102 | sequence_message.set_destination_channel("paxos_log_"); 103 | 104 | for (uint32 i = 1; i < participants_.size(); i++) { 105 | sequence_message.set_destination_node(participants_[i]); 106 | connection_->Send(sequence_message); 107 | } 108 | 109 | uint64 acks = 1; 110 | 111 | // Collect Acks. 112 | MessageProto message; 113 | while (acks < quorum) { 114 | while (connection_->GotMessage("paxos_log_", &message) == false) { 115 | usleep(10); 116 | if (!go_) { 117 | return; 118 | } 119 | } 120 | 121 | CHECK(message.type() == MessageProto::PAXOS_DATA_ACK); 122 | if (message.misc_int(0) == version) { 123 | acks++; 124 | } 125 | message.Clear(); 126 | } 127 | 128 | // Send the order to the locking thread 129 | sequence_message.set_type(MessageProto::PAXOS_BATCH_ORDER); 130 | sequence_message.set_destination_channel("scheduler_"); 131 | for (uint64 i = local_replica * machines_per_replica; i < (local_replica + 1)*machines_per_replica ;i++) { 132 | //LOG(ERROR) <Send(sequence_message); 135 | } 136 | 137 | sequence_message.clear_data(); 138 | 139 | // Commit! 140 | sequence_message.set_type(MessageProto::PAXOS_COMMIT); 141 | sequence_message.set_destination_channel("paxos_log_"); 142 | for (uint32 i = 1; i < participants_.size(); i++) { 143 | sequence_message.set_destination_node(participants_[i]); 144 | connection_->Send(sequence_message); 145 | } 146 | 147 | sequence_message.Clear(); 148 | //LOG(ERROR) << "In paxos log: append a sequence: "<Append(version, encoded); 150 | 151 | } 152 | } 153 | 154 | void Paxos::RunFollower() { 155 | 156 | MessageProto message; 157 | MessageProto ack_message; 158 | MessageProto append_message; 159 | queue uncommitted; 160 | 161 | uint64 machines_per_replica = configuration_->nodes_per_replica(); 162 | uint32 local_replica = configuration_->local_replica_id(); 163 | 164 | while (go_) { 165 | // Get message from leader. 166 | while (connection_->GotMessage("paxos_log_", &message) == false) { 167 | usleep(20); 168 | if (!go_) { 169 | return; 170 | } 171 | } 172 | 173 | if (message.type() == MessageProto::PAXOS_DATA) { 174 | // New proposal. 175 | uncommitted.push(message); 176 | // Send ack to leader. 177 | ack_message.set_destination_node(participants_[0]); 178 | ack_message.set_type(MessageProto::PAXOS_DATA_ACK); 179 | ack_message.set_destination_channel("paxos_log_"); 180 | ack_message.add_misc_int(message.misc_int(0)); 181 | connection_->Send(ack_message); 182 | 183 | ack_message.Clear(); 184 | 185 | } else if (message.type() == MessageProto::PAXOS_COMMIT){ 186 | // Commit message. 187 | CHECK(!uncommitted.empty()); 188 | append_message = uncommitted.front(); 189 | uncommitted.pop(); 190 | 191 | uint64 version = append_message.misc_int(0); 192 | string data = append_message.data(0); 193 | 194 | // Send the order to the locking thread 195 | append_message.set_type(MessageProto::PAXOS_BATCH_ORDER); 196 | append_message.set_destination_channel("scheduler_"); 197 | for (uint64 i = local_replica * machines_per_replica; i < (local_replica + 1)*machines_per_replica ;i++) { 198 | append_message.set_destination_node(i); 199 | connection_->Send(append_message); 200 | //LOG(ERROR) <Append(version, data); 206 | } 207 | } 208 | } 209 | 210 | -------------------------------------------------------------------------------- /src/log/paxos.h: -------------------------------------------------------------------------------- 1 | // Author: Kun Ren 2 | // 3 | #ifndef CALVIN_LOG_PAXOS_H_ 4 | #define CALVIN_LOG_PAXOS_H_ 5 | 6 | #include 7 | #include 8 | #include 9 | #include 10 | #include 11 | #include 12 | 13 | #include "proto/scalar.pb.h" 14 | #include "common/mutex.h" 15 | #include "common/types.h" 16 | #include "log/local_mem_log.h" 17 | #include "proto/scalar.pb.h" 18 | #include "machine/connection.h" 19 | 20 | using std::vector; 21 | using std::atomic; 22 | using std::make_pair; 23 | using std::pair; 24 | using std::queue; 25 | using std::set; 26 | 27 | 28 | class Paxos { 29 | public: 30 | Paxos(Log* log, ClusterConfig* config, ConnectionMultiplexer* connection); 31 | 32 | ~Paxos(); 33 | 34 | void Stop(); 35 | void Append(uint64 blockid); 36 | 37 | private: 38 | 39 | // Functions to start the Multiplexor's main loops, called in new pthreads by 40 | // the Sequencer's constructor. 41 | static void* RunLeaderThread(void *arg); 42 | static void* RunFollowerThread(void *arg); 43 | 44 | // Returns true iff leader. 45 | bool IsLeader(); 46 | 47 | // Leader's main loop. 48 | void RunLeader(); 49 | 50 | // Followers' main loop. 51 | void RunFollower(); 52 | 53 | // Participant list. 54 | vector participants_; 55 | 56 | // True iff main thread SHOULD run. 57 | bool go_; 58 | 59 | // Current request sequence that will get replicated. 60 | Sequence sequence_; 61 | std::atomic count_; 62 | Mutex mutex_; 63 | 64 | Log* log_; 65 | ClusterConfig* configuration_; 66 | uint64 this_machine_id_; 67 | 68 | ConnectionMultiplexer* connection_; 69 | 70 | // Separate pthread contexts in which to run the leader or follower thread. 71 | pthread_t leader_thread_; 72 | pthread_t follower_thread_; 73 | 74 | }; 75 | 76 | #endif // CALVIN_LOG_PAXOS_H_ 77 | -------------------------------------------------------------------------------- /src/machine/Makefile.inc: -------------------------------------------------------------------------------- 1 | # Create template specific variables 2 | UC_DIR := MACHINE 3 | LC_DIR := machine 4 | 5 | SRCS := machine/cluster_config.cc \ 6 | machine/cluster_manager.cc \ 7 | machine/connection.cc \ 8 | machine/sequencer.cc \ 9 | machine/lowlatency_sequencer.cc 10 | 11 | EXES := 12 | 13 | DEPS := $(PROTO_OBJS) $(COMMON_OBJS) $(LOG_OBJS) $(BACKEND_OBJS) 14 | 15 | # Link the template to avoid redundancy 16 | include $(MAKEFILE_TEMPLATE) 17 | -------------------------------------------------------------------------------- /src/machine/client.h: -------------------------------------------------------------------------------- 1 | // Author: Kun Ren 2 | // 3 | 4 | 5 | #ifndef _DB_MACHINE_CLIENT_H_ 6 | #define _DB_MACHINE_CLIENT_H_ 7 | 8 | #include 9 | #include 10 | #include 11 | #include 12 | #include 13 | #include 14 | 15 | #include "machine/cluster_config.h" 16 | #include "common/utils.h" 17 | #include "proto/txn.pb.h" 18 | #include "applications/microbenchmark.h" 19 | #include "applications/tpcc.h" 20 | 21 | using std::set; 22 | using std::string; 23 | using std::queue; 24 | using std::map; 25 | 26 | #define SAMPLES 180 27 | #define SAMPLE_RATE 199 28 | 29 | #define LATENCY_TEST 30 | 31 | #ifdef LATENCY_TEST 32 | extern map sequencer_recv; 33 | extern map scheduler_unlock; 34 | extern vector measured_latency; 35 | extern std::atomic latency_counter; 36 | #endif 37 | 38 | class ClusterConfig; 39 | class TxnProto; 40 | 41 | // Client 42 | class Client { 43 | public: 44 | virtual ~Client() {} 45 | virtual void GetTxn(TxnProto** txn, int txn_id) = 0; 46 | }; 47 | 48 | // Microbenchmark load generation client. 49 | class MClient : public Client { 50 | public: 51 | MClient(ClusterConfig* config, uint32 mp, uint32 hot_records) 52 | : microbenchmark(config, hot_records), config_(config), percent_mp_(mp), 53 | nodes_per_replica_(config->nodes_per_replica()), replative_node_id_(config->relative_node_id()) { 54 | } 55 | virtual ~MClient() {} 56 | virtual void GetTxn(TxnProto** txn, int txn_id) { 57 | if (nodes_per_replica_ > 1 && (uint32)(rand() % 100) < percent_mp_) { 58 | // Multipartition txn. 59 | uint64 other; 60 | do { 61 | other = (uint64)(rand() % nodes_per_replica_); 62 | } while (other == replative_node_id_); 63 | *txn = microbenchmark.MicroTxnMP(txn_id, replative_node_id_, other); 64 | } else { 65 | // Single-partition txn. 66 | *txn = microbenchmark.MicroTxnSP(txn_id, replative_node_id_); 67 | } 68 | } 69 | 70 | private: 71 | Microbenchmark microbenchmark; 72 | ClusterConfig* config_; 73 | uint32 percent_mp_; 74 | uint64 nodes_per_replica_; 75 | uint64 replative_node_id_; 76 | }; 77 | 78 | // TPCC load generation client. 79 | class TClient : public Client { 80 | public: 81 | TClient(ClusterConfig* config, uint32 mp, uint32 hot_records) 82 | : tpcc(config, hot_records), config_(config), percent_mp_(mp), 83 | nodes_per_replica_(config->nodes_per_replica()), replative_node_id_(config->relative_node_id()) { 84 | } 85 | virtual ~TClient() {} 86 | virtual void GetTxn(TxnProto** txn, int txn_id) { 87 | // Right now only test 10% multi-warehouse txn (percent_mp_ is used to how much multi-warehouse txn) 88 | if (nodes_per_replica_ > 1 && (uint32)(rand() % 100) < percent_mp_) { 89 | // Multipartition txn. 90 | uint64 other; 91 | do { 92 | other = (uint64)(rand() % nodes_per_replica_); 93 | } while (other == replative_node_id_); 94 | *txn = tpcc.TpccTxnMP(txn_id, replative_node_id_, other); 95 | } else { 96 | // Single-partition txn. 97 | *txn = tpcc.TpccTxnSP(txn_id, replative_node_id_); 98 | } 99 | } 100 | 101 | private: 102 | Tpcc tpcc; 103 | ClusterConfig* config_; 104 | uint32 percent_mp_; 105 | uint64 nodes_per_replica_; 106 | uint64 replative_node_id_; 107 | }; 108 | 109 | // Microbenchmark load generation client. 110 | class Lowlatency_MClient : public Client { 111 | public: 112 | Lowlatency_MClient(ClusterConfig* config, uint32 mp, uint32 mr, uint32 hot_records) 113 | : microbenchmark(config, hot_records), config_(config), percent_mp_(mp), percent_mr_(mr), 114 | nodes_per_replica_(config->nodes_per_replica()), replative_node_id_(config->relative_node_id()) { 115 | local_replica_ = config_->local_replica_id(); 116 | num_replicas_ = config_->replicas_size(); 117 | } 118 | virtual ~Lowlatency_MClient() {} 119 | virtual void GetTxn(TxnProto** txn, int txn_id) { 120 | if ((uint32)(rand() % 100) < percent_mr_) { 121 | // Multi-replica txn. 122 | uint32 other_replica; 123 | do { 124 | other_replica = (uint32)(rand() % num_replicas_); 125 | } while (other_replica == local_replica_); 126 | //other_replica = (local_replica_ + 1)%3; 127 | if (nodes_per_replica_ > 1 && uint32(rand() % 100) < percent_mp_) { 128 | // Multi-replica multi-partition txn 129 | uint64 other_node; 130 | do { 131 | other_node = (uint64)(rand() % nodes_per_replica_); 132 | } while (other_node == replative_node_id_); 133 | 134 | *txn = microbenchmark.MicroTxnMRMP(txn_id, replative_node_id_, other_node, local_replica_, other_replica); 135 | //*txn = microbenchmark.MicroTxnMRMP(txn_id, 0, 1, 0, 1); 136 | } else { 137 | // Multi-replica single-partition txn 138 | *txn = microbenchmark.MicroTxnMRSP(txn_id, replative_node_id_, local_replica_, other_replica); 139 | //*txn = microbenchmark.MicroTxnMRSP(txn_id, 0, 0, 1); 140 | } 141 | } else { 142 | // Single-replica txn. 143 | if (nodes_per_replica_ > 1 && (uint32)(rand() % 100) < percent_mp_) { 144 | // Single-replica multi-partition txn 145 | uint64 other_node; 146 | do { 147 | other_node = (uint64)(rand() % nodes_per_replica_); 148 | } while (other_node == replative_node_id_); 149 | 150 | *txn = microbenchmark.MicroTxnSRMP(txn_id, replative_node_id_, other_node, local_replica_); 151 | } else { 152 | // Single-replica single-partition txn 153 | *txn = microbenchmark.MicroTxnSRSP(txn_id, replative_node_id_, local_replica_); 154 | } 155 | } 156 | } 157 | 158 | private: 159 | Microbenchmark microbenchmark; 160 | ClusterConfig* config_; 161 | uint32 percent_mp_; 162 | uint32 percent_mr_; 163 | uint32 local_replica_; 164 | uint32 num_replicas_; 165 | uint64 nodes_per_replica_; 166 | uint64 replative_node_id_; 167 | }; 168 | 169 | // Tpcc load generation client for slog. 170 | class Lowlatency_TClient : public Client { 171 | public: 172 | Lowlatency_TClient(ClusterConfig* config, uint32 mp, uint32 mr, uint32 hot_records) 173 | : tpcc(config, hot_records), config_(config), percent_mp_(mp), percent_mr_(mr), 174 | nodes_per_replica_(config->nodes_per_replica()), replative_node_id_(config->relative_node_id()) { 175 | local_replica_ = config_->local_replica_id(); 176 | num_replicas_ = config_->replicas_size(); 177 | } 178 | virtual ~Lowlatency_TClient() {} 179 | virtual void GetTxn(TxnProto** txn, int txn_id) { 180 | // Currently use 10% multi-warehouse txn (percent_mp_ is used to how many multi-warehouse txn) 181 | if ((uint32)(rand() % 100) < percent_mp_) { 182 | if ((uint32)(rand() % 100) < percent_mr_) { 183 | // Multi-replica txn. 184 | uint32 other_replica; 185 | do { 186 | other_replica = (uint32)(rand() % num_replicas_); 187 | } while (other_replica == local_replica_); 188 | // Multi-replica multi-partition txn 189 | uint64 other_node; 190 | do { 191 | other_node = (uint64)(rand() % nodes_per_replica_); 192 | } while (other_node == replative_node_id_); 193 | 194 | *txn = tpcc.TpccTxnMRMP(txn_id, replative_node_id_, other_node, local_replica_, other_replica); 195 | } else { 196 | // Single-replica txn. 197 | // Single-replica multi-partition txn 198 | uint64 other_node; 199 | do { 200 | other_node = (uint64)(rand() % nodes_per_replica_); 201 | } while (other_node == replative_node_id_); 202 | 203 | *txn = tpcc.TpccTxnSRMP(txn_id, replative_node_id_, other_node, local_replica_); 204 | } 205 | } else { 206 | // Single-replica single-partition txn 207 | *txn = tpcc.TpccTxnSRSP(txn_id, replative_node_id_, local_replica_); 208 | } 209 | } 210 | 211 | private: 212 | Tpcc tpcc; 213 | ClusterConfig* config_; 214 | uint32 percent_mp_; 215 | uint32 percent_mr_; 216 | uint32 local_replica_; 217 | uint32 num_replicas_; 218 | uint64 nodes_per_replica_; 219 | uint64 replative_node_id_; 220 | }; 221 | 222 | 223 | #endif // _DB_MACHINE_CLIENT_H_ 224 | -------------------------------------------------------------------------------- /src/machine/cluster_config.cc: -------------------------------------------------------------------------------- 1 | // Author: Kun Ren 2 | // 3 | 4 | #include "machine/cluster_config.h" 5 | 6 | #include 7 | #include 8 | #include 9 | #include 10 | #include 11 | #include 12 | #include 13 | 14 | 15 | using std::map; 16 | using std::set; 17 | using std::string; 18 | using std::ifstream; 19 | using std::ostringstream; 20 | 21 | uint64 ClusterConfig::HashBatchID(uint64 batch_id) { 22 | return FNVHash(UInt64ToString(33 * batch_id)) % nodes_per_replica(); 23 | } 24 | 25 | uint64 ClusterConfig::LookupMachineID(uint64 relative_id, uint64 replica) { 26 | return replica * nodes_per_replica() + relative_id; 27 | } 28 | 29 | // TODO(kun): Implement better (application-specific?) partitioning. 30 | uint64 ClusterConfig::LookupPartition(const Key& key) { 31 | // For microbenchmark 32 | return StringToInt(key) % nodes_per_replica(); 33 | } 34 | 35 | uint32 ClusterConfig::LookupMaster(const Key& key) { 36 | // For microbenchmark 37 | return (StringToInt(key) / nodes_per_replica()) % replicas_size_; 38 | } 39 | 40 | // Checks to see if a config string appears to be a valid cluster config repr. 41 | void CheckString(const string& config) { 42 | // Track machine ids and host-port pairs that we have already seen to ensure 43 | // that none are repeated. 44 | set machine_ids; 45 | set host_port_pairs; 46 | 47 | // Split config string into lines. 48 | vector entries = SplitString(config, '\n'); 49 | for (uint32 i = 0; i < entries.size(); i++) { 50 | // Skip empty lines. 51 | if (!entries[i].empty()) { 52 | // Each entry should consist of three colon-delimited parts: id, host and 53 | // port. Parse entry and check validity of id/host/port. 54 | vector entry = SplitString(entries[i], ':'); 55 | CHECK(entry.size() == 4) << "bad config line: " << entries[i]; 56 | uint64 id = static_cast(StringToInt(entry[0])); 57 | uint32 replica = static_cast(StringToInt(entry[1])); 58 | string host = entry[2]; 59 | int port = StringToInt(entry[3]); 60 | CHECK(static_cast(id) >= 0) 61 | << "bad machine id: " << static_cast(id); 62 | CHECK(static_cast(replica) >= 0) 63 | << "bad replica id: " << static_cast(replica); 64 | CHECK(host.size() > 0) << "empty hostname"; 65 | CHECK(port >= 0) << "bad port: " << port; 66 | 67 | // Check for repeated machine ids. 68 | CHECK(machine_ids.count(id) == 0) 69 | << "repeated machine id: " << id; 70 | machine_ids.insert(id); 71 | 72 | 73 | // Check for repeated host/port pairs. 74 | string hostport = host + ":" + IntToString(port); 75 | CHECK(host_port_pairs.count(hostport) == 0) 76 | << "repeated host/port pair: " << hostport; 77 | host_port_pairs.insert(hostport); 78 | } 79 | } 80 | } 81 | 82 | // Checks to see if a config proto appears to be a valid cluster config repr. 83 | void CheckProto(const ClusterConfigProto& config) { 84 | // Track machine ids and host-port pairs that we have already seen to ensure 85 | // that none are repeated. 86 | set machine_ids; 87 | set host_port_pairs; 88 | 89 | for (int i = 0; i < config.machines_size(); i++) { 90 | // Check validity of id/host/port. 91 | CHECK(config.machines(i).has_id()) << "missing machind id"; 92 | CHECK(config.machines(i).has_replica()) << "missing replica id"; 93 | CHECK(config.machines(i).has_host())<< "missing host"; 94 | CHECK(config.machines(i).has_port())<< "missing port"; 95 | CHECK(static_cast(config.machines(i).id()) >= 0) 96 | << "bad machine id: " << static_cast(config.machines(i).id()); 97 | CHECK(static_cast(config.machines(i).replica()) >= 0) 98 | << "bad replica id: " << static_cast(config.machines(i).replica()); 99 | CHECK(config.machines(i).host().size() > 0) 100 | << "empty hostname"; 101 | CHECK(config.machines(i).port() >= 0) 102 | << "bad port: " << config.machines(i).port(); 103 | 104 | // Check for repeated machine ids. 105 | CHECK(machine_ids.count(config.machines(i).id()) == 0) 106 | << "repeated machine id: " << config.machines(i).id(); 107 | machine_ids.insert(config.machines(i).id()); 108 | 109 | // Check for repeated host/port pairs. 110 | string hostport = config.machines(i).host() + ":" + 111 | IntToString(config.machines(i).port()); 112 | CHECK(host_port_pairs.count(hostport) == 0) 113 | << "repeated host/port pair: " << hostport; 114 | host_port_pairs.insert(hostport); 115 | } 116 | } 117 | 118 | 119 | void ClusterConfig::FromFile(const string& filename) { 120 | string config; 121 | 122 | ifstream ifile(filename.c_str()); 123 | ostringstream buf; 124 | char ch; 125 | while(buf&&ifile.get(ch)) { 126 | buf.put(ch); 127 | } 128 | 129 | config = buf.str(); 130 | 131 | FromString(config); 132 | } 133 | 134 | void ClusterConfig::FromString(const string& config) { 135 | CheckString(config); 136 | 137 | // Clear any previous machine information. 138 | machines_.clear(); 139 | machines_replica_.clear(); 140 | set replicas; 141 | 142 | // Split config string into lines. 143 | vector entries = SplitString(config, '\n'); 144 | for (uint32 i = 0; i < entries.size(); i++) { 145 | // Skip empty lines. 146 | if (!entries[i].empty()) { 147 | // Each entry should consist of three colon-delimited parts: id, host and 148 | // port. Parse entry and check validity of id/host/port. 149 | vector entry = SplitString(entries[i], ':'); 150 | 151 | // Add entry. 152 | uint64 id = static_cast(StringToInt(entry[0])); 153 | uint32 replica = static_cast(StringToInt(entry[1])); 154 | 155 | machines_[id].set_id(id); 156 | machines_[id].set_replica(replica); 157 | machines_[id].set_host(entry[2]); 158 | machines_[id].set_port(StringToInt(entry[3])); 159 | 160 | machines_replica_[id] = replica; 161 | replicas.insert(replica); 162 | 163 | if (id == local_node_id_) { 164 | local_replica_ = replica; 165 | } 166 | } 167 | } 168 | 169 | replicas_size_ = replicas.size(); 170 | relative_node_id_ = local_node_id_ % (machines_.size() / replicas_size_); 171 | } 172 | 173 | void ClusterConfig::FromProto(const ClusterConfigProto& config) { 174 | machines_.clear(); 175 | machines_replica_.clear(); 176 | set replicas; 177 | 178 | for (int i = 0; i < config.machines_size(); i++) { 179 | machines_[config.machines(i).id()] = config.machines(i); 180 | machines_replica_[config.machines(i).id()] = config.machines(i).replica(); 181 | replicas.insert(config.machines(i).replica()); 182 | 183 | if (config.machines(i).id() == local_node_id_) { 184 | local_replica_ = config.machines(i).replica(); 185 | } 186 | } 187 | 188 | replicas_size_ = replicas.size(); 189 | relative_node_id_ = local_node_id_ % (machines_.size() / replicas_size_); 190 | } 191 | 192 | 193 | -------------------------------------------------------------------------------- /src/machine/cluster_config.h: -------------------------------------------------------------------------------- 1 | // Author: Kun Ren 2 | // 3 | // A ClusterConfig represents a local view of a collection of Machines. 4 | // Constructing a ClusterConfig object does not actually deploy an application 5 | // across the collection of physical servers (use ClusterManager for that). 6 | // 7 | 8 | 9 | #ifndef CALVIN_MACHINE_CLUSTER_CONFIG_H_ 10 | #define CALVIN_MACHINE_CLUSTER_CONFIG_H_ 11 | 12 | #include 13 | #include 14 | #include 15 | 16 | #include "common/types.h" 17 | #include "common/utils.h" 18 | #include "proto/cluster_config.pb.h" 19 | 20 | using std::map; 21 | using std::string; 22 | 23 | class ClusterConfig { 24 | public: 25 | // Default constructor creates a ClusterConfig consisting of no Machines. 26 | // Note that this is completely useless until 'FromFile()' or 'FromString()' 27 | // is called to populate it with information about the actual cluster 28 | // configuration. 29 | ClusterConfig(uint64 local_node_id): local_node_id_(local_node_id), next_guid_(1000), stop_(false) {} 30 | 31 | ClusterConfig(): local_node_id_(0), next_guid_(1000), stop_(false) {} 32 | 33 | ~ClusterConfig() {Stop();} 34 | 35 | // Populates a ClusterConfig using a specification consisting of zero or 36 | // more (newline delimited) lines of the format: 37 | // 38 | // ::: 39 | // 40 | // The specification can either be read from a file, provided as a string, 41 | // or provided as a protobuf (see proto/cluster_config.proto). 42 | // 43 | // Each MachineID that appears in the specification must be unique, as must 44 | // each (ip, port) pair. 45 | void FromFile(const string& filename); 46 | void FromString(const string& config); 47 | void FromProto(const ClusterConfigProto& config); 48 | 49 | // ClusterConfigs can also be written out to files, strings, or protos. 50 | void ToFile(const string& filename); 51 | void ToString(string* out); 52 | void ToProto(ClusterConfigProto* out); 53 | 54 | uint64 HashBatchID(uint64 batch_id); 55 | 56 | uint64 LookupMachineID(uint64 relative_id, uint64 replica); 57 | 58 | uint64 LookupPartition(const Key& key); 59 | 60 | uint32 LookupMaster(const Key& key); 61 | 62 | // Returns the number of machines that appear in the config. 63 | inline uint64 all_nodes_size() const { 64 | return static_cast(machines_.size()); 65 | } 66 | 67 | inline uint64 nodes_per_replica() const { 68 | return static_cast(machines_.size()/replicas_size_); 69 | } 70 | 71 | inline uint64 local_node_id() const { 72 | return local_node_id_; 73 | } 74 | 75 | inline uint64 relative_node_id() const { 76 | return relative_node_id_; 77 | } 78 | 79 | inline uint32 replicas_size() const { 80 | return replicas_size_; 81 | } 82 | 83 | inline uint32 local_replica_id() const { 84 | return local_replica_; 85 | } 86 | 87 | inline uint32 LookupReplica(uint64 machine_id) const { 88 | return machine_id / nodes_per_replica(); 89 | } 90 | 91 | bool Stopped() { 92 | return stop_; 93 | } 94 | 95 | void Stop() { 96 | stop_ = true; 97 | } 98 | 99 | // Returns a globally unique ID (no ordering guarantees though). 100 | uint64 GetGUID() { 101 | Lock l(&mutex_); 102 | return 1 + local_node_id_ + (all_nodes_size() * (next_guid_++)); 103 | } 104 | 105 | // Returns true and populates '*info' accordingly iff the config tracks a 106 | // machine with id 'id'. 107 | inline bool lookup_machine(uint64 id, MachineInfo* info) { 108 | if (machines_.count(id)) { 109 | *info = machines_[id]; 110 | return true; 111 | } 112 | return false; 113 | } 114 | 115 | // Returns a const ref to the underlying collection of machine records. 116 | const map& machines() { 117 | return machines_; 118 | } 119 | 120 | // Contains all machines. 121 | map machines_; 122 | 123 | private: 124 | 125 | map machines_replica_; 126 | 127 | uint32 replicas_size_; 128 | 129 | uint64 local_node_id_; 130 | 131 | uint32 local_replica_; 132 | 133 | uint64 relative_node_id_; 134 | 135 | // Globally unique ID source. 136 | std::atomic next_guid_; 137 | 138 | // True iff machine has received an external 'stop' request and the server 139 | // needs to exit gracefully. 140 | bool stop_; 141 | // Intentionally copyable. 142 | 143 | Mutex mutex_; 144 | }; 145 | 146 | #endif // CALVIN_MACHINE_CLUSTER_CONFIG_H_ 147 | 148 | -------------------------------------------------------------------------------- /src/machine/cluster_manager.cc: -------------------------------------------------------------------------------- 1 | // Author: Kun Ren 2 | // 3 | 4 | #include 5 | #include 6 | #include 7 | #include 8 | 9 | #include 10 | #include 11 | #include 12 | #include 13 | #include 14 | 15 | #include "machine/cluster_manager.h" 16 | #include "common/utils.h" 17 | 18 | using std::string; 19 | 20 | const string& ClusterManager::ssh_key(uint64 m) { 21 | if (config_.all_nodes_size() < 3) { 22 | return ssh_key1_; 23 | } 24 | int repsize = config_.all_nodes_size() / num_replicas_; 25 | 26 | if (m / repsize == 0) { 27 | return ssh_key1_; 28 | } else if (m / repsize == 1) { 29 | return ssh_key2_; 30 | } else if (m / repsize == 2) { 31 | return ssh_key3_; 32 | } else if (m / repsize == 3) { 33 | return ssh_key4_; 34 | } else if (m / repsize == 4) { 35 | return ssh_key5_; 36 | } else if (m / repsize == 5) { 37 | return ssh_key6_; 38 | } 39 | 40 | LOG(FATAL) << "bad machine id: " << m; 41 | } 42 | 43 | void* SystemFunction(void* arg) { 44 | // Run the specified command. 45 | int status = system(reinterpret_cast(arg)->c_str()); 46 | if(status == -1){ 47 | LOG(FATAL)<<"system error"; 48 | } else if(WIFEXITED(status) && (WEXITSTATUS(status) == 0)){ 49 | // printf("run command successful\n"); 50 | } else { 51 | LOG(FATAL) << "run command fail and exit code is " << WEXITSTATUS(status); 52 | } 53 | 54 | delete reinterpret_cast(arg); 55 | return NULL; 56 | } 57 | 58 | void ClusterManager::PutConfig() { 59 | // Copy config file to all machines. 60 | vector threads; 61 | for (map::const_iterator it = 62 | config_.machines().begin(); 63 | it != config_.machines().end(); ++it) { 64 | threads.resize(threads.size()+1); 65 | string* ssh_command = new string( 66 | "scp " + ssh_key(it->first) + " " + config_file_ + 67 | " "+ ssh_username_ + "@" + it->second.host() + ":" + calvin_path_ 68 | + "/" + config_file_); 69 | pthread_create( 70 | &threads[threads.size()-1], 71 | NULL, 72 | SystemFunction, 73 | reinterpret_cast(ssh_command)); 74 | } 75 | for (uint32 i = 0; i < threads.size(); i++) { 76 | pthread_join(threads[i], NULL); 77 | } 78 | } 79 | 80 | void ClusterManager::GetTempFiles(const string& base) { 81 | vector threads; 82 | for (map::const_iterator it = 83 | config_.machines().begin(); 84 | it != config_.machines().end(); ++it) { 85 | threads.resize(threads.size()+1); 86 | string* ssh_command = new string( 87 | "scp " + ssh_key(it->first) + " "+ ssh_username_ + "@" + it->second.host() + 88 | ":/tmp/" + base + IntToString(threads.size()-1) + " data/"); 89 | pthread_create( 90 | &threads[threads.size()-1], 91 | NULL, 92 | SystemFunction, 93 | reinterpret_cast(ssh_command)); 94 | } 95 | for (uint32 i = 0; i < threads.size(); i++) { 96 | pthread_join(threads[i], NULL); 97 | } 98 | } 99 | 100 | void ClusterManager::Update() { 101 | // Next, Run "git pull ;make clean;make -j" to get the latest code and compile. 102 | vector threads; 103 | for (map::const_iterator it = 104 | config_.machines().begin(); 105 | it != config_.machines().end(); ++it) { 106 | threads.resize(threads.size()+1); 107 | string* ssh_command = new string( 108 | "ssh " + ssh_key(it->first) + " "+ ssh_username_ + "@" + it->second.host() + 109 | " 'cd " + calvin_path_ + ";git checkout calvin.conf; git pull; cd src; cp Makefile.default Makefile; make clean; make -j'"); 110 | pthread_create( 111 | &threads[threads.size()-1], 112 | NULL, 113 | SystemFunction, 114 | reinterpret_cast(ssh_command)); 115 | } 116 | for (uint32 i = 0; i < threads.size(); i++) { 117 | pthread_join(threads[i], NULL); 118 | } 119 | threads.clear(); 120 | } 121 | 122 | void ClusterManager::DeployCluster(int experiment, int percent_mp, int percent_mr, int hot_records, int max_batch_size) { 123 | vector threads; 124 | // Now ssh into all machines and start 'binary' running. 125 | for (map::const_iterator it = 126 | config_.machines().begin(); 127 | it != config_.machines().end(); ++it) { 128 | string val; 129 | threads.resize(threads.size()+1); 130 | string* ssh_command = new string( 131 | "ssh " + ssh_key(it->first) + " "+ ssh_username_ + "@" + it->second.host() + 132 | " 'cd " + calvin_path_ + "; " + " bin/scripts/" + binary_ + 133 | " --machine_id=" + IntToString(it->second.id()) + " --mode=" + IntToString(mode_) + " --type=" + IntToString(type_) + 134 | " --config=" + config_file_ + " --experiment=" + IntToString(experiment) + " --percent_mp=" + IntToString(percent_mp) + " --percent_mr=" + IntToString(percent_mr) + 135 | " --hot_records=" + IntToString(hot_records) + " --max_batch_size=" + IntToString(max_batch_size) + " ' &"); 136 | 137 | pthread_create( 138 | &threads[threads.size()-1], 139 | NULL, 140 | SystemFunction, 141 | reinterpret_cast(ssh_command)); 142 | } 143 | for (uint32 i = 0; i < threads.size(); i++) { 144 | pthread_join(threads[i], NULL); 145 | } 146 | } 147 | 148 | void ClusterManager::KillCluster() { 149 | vector threads; 150 | 151 | for (map::const_iterator it = 152 | config_.machines().begin(); 153 | it != config_.machines().end(); ++it) { 154 | threads.resize(threads.size()+1); 155 | string* ssh_command = new string( 156 | "ssh " + ssh_key(it->first) + " " + ssh_username_ + "@" + it->second.host() + 157 | " killall -9 " + binary_); 158 | pthread_create( 159 | &threads[threads.size()-1], 160 | NULL, 161 | SystemFunction, 162 | reinterpret_cast(ssh_command)); 163 | } 164 | for (uint32 i = 0; i < threads.size(); i++) { 165 | pthread_join(threads[i], NULL); 166 | } 167 | } 168 | 169 | void ClusterManager::ClusterStatus() { 170 | // 0: Unreachable 171 | // 1: Calvin not found 172 | // 2: Running 173 | // 3: Not Running 174 | vector cluster_status(config_.machines().size()); 175 | int index = 0; 176 | 177 | for (map::const_iterator it = 178 | config_.machines().begin(); 179 | it != config_.machines().end(); ++it) { 180 | uint64 machine_id = it->second.id(); 181 | string host = it->second.host(); 182 | 183 | // Same stuff with DeployCluster 184 | string ssh_command = "ssh " + ssh_key(it->first) + " " + ssh_username_ + "@" + host 185 | + " 'cd " + calvin_path_ + "; bin/scripts/" + binary_ 186 | + " --calvin_version=true" + " --machine_id=" 187 | + IntToString(machine_id) + "'"; 188 | int status = system(ssh_command.c_str()); 189 | if (status == -1 || WIFEXITED(status) == false || 190 | WEXITSTATUS(status) != 0) { 191 | if (WEXITSTATUS(status) == 255 || WIFEXITED(status) == false) { 192 | cluster_status[index++] = 0; 193 | } else if (WEXITSTATUS(status) == 127) { 194 | cluster_status[index++] = 1; 195 | } else if (WEXITSTATUS(status) == 254) { 196 | cluster_status[index++] = 2; 197 | } 198 | } else { 199 | cluster_status[index++] = 3; 200 | } 201 | } 202 | 203 | printf("----------------------Cluster Status-----------------------------\n"); 204 | printf("machine id host:port status \n"); 205 | index = 0; 206 | for (map::const_iterator it = 207 | config_.machines().begin(); 208 | it != config_.machines().end(); ++it) { 209 | uint64 machine_id = it->second.id(); 210 | string host = it->second.host(); 211 | int port = it->second.port(); 212 | 213 | switch (cluster_status[index++]) { 214 | case 0: 215 | printf("%-10d %16s:%d Unreachable\n", 216 | (int)machine_id, host.c_str(), port); 217 | break; 218 | case 1: 219 | printf("%-10d %16s:%d Calvin not found\n", 220 | (int)machine_id, host.c_str(), port); 221 | break; 222 | case 2: 223 | printf("%-10d %16s:%d Running\n", 224 | (int)machine_id, host.c_str(), port); 225 | break; 226 | case 3: 227 | printf("%-10d %16s:%d Not Running\n", 228 | (int)machine_id, host.c_str(), port); 229 | break; 230 | default: 231 | break; 232 | } 233 | } 234 | printf("-----------------------------------------------------------------\n"); 235 | } 236 | 237 | const ClusterConfig& ClusterManager::GetConfig() { 238 | return config_; 239 | } 240 | 241 | -------------------------------------------------------------------------------- /src/machine/cluster_manager.h: -------------------------------------------------------------------------------- 1 | // Author: Kun Ren 2 | // 3 | // A ClusterManager is a tool for deploying, tracking, maintaining, modifying, 4 | // and tearing down machine clusters. 5 | // 6 | // 7 | 8 | #ifndef CALVIN_MACHINE_CLUSTER_MANAGER_H_ 9 | #define CALVIN_MACHINE_CLUSTER_MANAGER_H_ 10 | 11 | #include 12 | 13 | #include "machine/cluster_config.h" 14 | 15 | 16 | using std::string; 17 | 18 | class ClusterManager { 19 | public: 20 | // Sets initial target config. 21 | ClusterManager(const string& config_file, const string& calvin_path, 22 | const string& binary, const uint32& mode, const uint32& type, 23 | const string& ssh_key1, const string& ssh_key2, const string& ssh_key3) 24 | : config_file_(config_file), calvin_path_(calvin_path), binary_(binary), mode_(mode), type_(type), 25 | ssh_username_("ubuntu"), ssh_key1_(ssh_key1), ssh_key2_(ssh_key2), ssh_key3_(ssh_key3) { 26 | config_.FromFile(config_file_); 27 | num_replicas_ = 3; 28 | } 29 | 30 | // Sets initial target config. 31 | ClusterManager(const string& config_file, const string& calvin_path, 32 | const string& binary, const uint32& mode, const uint32& type, 33 | const string& ssh_key1, const string& ssh_key2, const string& ssh_key3, 34 | const string& ssh_key4, const string& ssh_key5, const string& ssh_key6) 35 | : config_file_(config_file), calvin_path_(calvin_path), binary_(binary), mode_(mode), type_(type), 36 | ssh_username_("ubuntu"), ssh_key1_(ssh_key1), ssh_key2_(ssh_key2), ssh_key3_(ssh_key3), 37 | ssh_key4_(ssh_key4), ssh_key5_(ssh_key5), ssh_key6_(ssh_key6) { 38 | config_.FromFile(config_file_); 39 | num_replicas_ = 6; 40 | } 41 | 42 | ~ClusterManager() { 43 | } 44 | 45 | // Runs "svn up" and rebuilds calvin on every machine in the cluster. 46 | void Update(); 47 | 48 | // Attempts to deploy the cluster according to config.... 49 | // 50 | // First, performs several checks (and dies with a useful error message if 51 | // any of them fail): 52 | // - checks that all participants are reachable by ssh 53 | // - checks that all participants have calvin (with same version as server) 54 | // - checks that all participants are NOT already running calvin instances 55 | // 56 | // Next, Run "svn up;make clean;make -j" to get the latest code and compile. 57 | // 58 | // Finally, ssh into all machines and start 'binary' running. 59 | // 60 | // 61 | // TODO(kun): FUTURE WORK - don't implement now: 62 | // Also start a monitoring thread going that occasionally polls machines 63 | // in the cluster to generate cluster status reports, repair problems, etc. 64 | void DeployCluster(int experiment, int percent_mp, int percent_mr, int hot_records, int max_batch_size); 65 | 66 | // Kills all participating machine processes (using 'ssh killall', so they do 67 | // not need to exit gracefully). 68 | void KillCluster(); 69 | 70 | // Returns a human-readable report about cluster status including: 71 | // - what participants are currently unreachable by ssh (if any) 72 | // - what participants are reachable by ssh but NOT running an instance of 73 | // the server binary 74 | void ClusterStatus(); 75 | 76 | const ClusterConfig& GetConfig(); 77 | 78 | void PutConfig(); 79 | void GetTempFiles(const string& base); 80 | 81 | private: 82 | // Returns ssh key for machine m. 83 | const string& ssh_key(uint64 m); 84 | 85 | // Configuration of machines managed by this ClusterManager. 86 | ClusterConfig config_; 87 | 88 | string config_file_; 89 | 90 | string calvin_path_; 91 | 92 | string binary_; 93 | 94 | uint32 mode_; 95 | 96 | uint32 type_; 97 | 98 | // Username with which to ssh to machines. 99 | // Default: 'ubuntu' 100 | // TODO(kun): Make this more easily configurable. 101 | string ssh_username_; 102 | 103 | // For ssh authentication, used for EC2, if test on zoo, just set it " "; 104 | // If test on EC2, set it to "-i YOUR_KEY.pem" 105 | string ssh_key1_; 106 | string ssh_key2_; 107 | string ssh_key3_; 108 | string ssh_key4_; 109 | string ssh_key5_; 110 | string ssh_key6_; 111 | 112 | // Number of replicas 113 | uint32 num_replicas_; 114 | 115 | // DISALLOW_DEFAULT_CONSTRUCTOR 116 | ClusterManager(); 117 | 118 | // DISALLOW_COPY_AND_ASSIGN 119 | ClusterManager(const ClusterManager&); 120 | ClusterManager& operator=(const ClusterManager&); 121 | }; 122 | 123 | #endif // CALVIN_MACHINE_CLUSTER_MANAGER_H_ 124 | 125 | -------------------------------------------------------------------------------- /src/machine/connection.cc: -------------------------------------------------------------------------------- 1 | // Author: Kun Ren 2 | // 3 | 4 | #include "machine/connection.h" 5 | 6 | using zmq::socket_t; 7 | 8 | ConnectionMultiplexer::ConnectionMultiplexer(ClusterConfig* config) 9 | : configuration_(config), context_(1), deconstructor_invoked_(false) { 10 | local_node_id_ = config->local_node_id(); 11 | port_ = config->machines_.find(local_node_id_)->second.port(); 12 | 13 | // Bind port for remote incoming socket. 14 | char endpoint[256]; 15 | snprintf(endpoint, sizeof(endpoint), "tcp://*:%d", port_); 16 | remote_in_ = new socket_t(context_, ZMQ_PULL); 17 | remote_in_->bind(endpoint); 18 | 19 | link_unlink_queue_ = new AtomicQueue(); 20 | 21 | new_channel_queue_ = new AtomicQueue(); 22 | 23 | delete_channel_queue_ = new AtomicQueue(); 24 | 25 | send_message_queue_ = new AtomicQueue(); 26 | 27 | // Wait for other nodes to bind sockets before connecting to them. 28 | Spin(0.1); 29 | 30 | // Connect to remote outgoing sockets. 31 | for (map::const_iterator it = config->machines_.begin(); 32 | it != config->machines_.end(); ++it) { 33 | if (it->second.id() != local_node_id_) { // Only remote nodes. 34 | snprintf(endpoint, sizeof(endpoint), "tcp://%s:%d", 35 | it->second.host().c_str(), it->second.port()); 36 | remote_out_[it->second.id()] = new socket_t(context_, ZMQ_PUSH); 37 | remote_out_[it->second.id()]->connect(endpoint); 38 | } 39 | } 40 | 41 | cpu_set_t cpuset; 42 | pthread_attr_t attr; 43 | pthread_attr_init(&attr); 44 | 45 | CPU_ZERO(&cpuset); 46 | CPU_SET(3, &cpuset); 47 | //CPU_SET(4, &cpuset); 48 | //CPU_SET(5, &cpuset); 49 | //CPU_SET(6, &cpuset); 50 | //CPU_SET(7, &cpuset); 51 | pthread_attr_setaffinity_np(&attr, sizeof(cpu_set_t), &cpuset); 52 | 53 | 54 | // Start Multiplexer main loop running in background thread. 55 | pthread_create(&thread_, &attr, RunMultiplexer, reinterpret_cast(this)); 56 | 57 | // Just to be safe, wait a bit longer for all other nodes to finish 58 | // multiplexer initialization before returning to the caller, who may start 59 | // sending messages immediately. 60 | Spin(0.1); 61 | } 62 | 63 | ConnectionMultiplexer::~ConnectionMultiplexer() { 64 | // Stop the multixplexer's main loop. 65 | deconstructor_invoked_ = true; 66 | pthread_join(thread_, NULL); 67 | 68 | // Close tcp sockets. 69 | delete remote_in_; 70 | for (unordered_map::iterator it = remote_out_.begin(); 71 | it != remote_out_.end(); ++it) { 72 | delete it->second; 73 | } 74 | 75 | channel_results_.Destroy(); 76 | delete link_unlink_queue_; 77 | } 78 | 79 | 80 | bool ConnectionMultiplexer::GotMessage(const string& channel, MessageProto* message) { 81 | CHECK(channel_results_.Count(channel) > 0); 82 | 83 | if ((channel_results_.Lookup(channel))->Pop(message)) { 84 | return true; 85 | } else { 86 | return false; 87 | } 88 | } 89 | 90 | void ConnectionMultiplexer::NewChannel(const string& channel) { 91 | // Disallow concurrent calls to NewConnection/~Connection. 92 | new_channel_queue_->Push(channel); 93 | usleep(1000); 94 | while (channel_results_.Count(channel) == 0) { 95 | usleep(200); 96 | } 97 | 98 | CHECK(channel_results_.Count(channel) > 0); 99 | usleep(1000); 100 | } 101 | 102 | 103 | void ConnectionMultiplexer::DeleteChannel(const string& channel) { 104 | // Serve any pending (valid) connection deletion request. 105 | delete_channel_queue_->Push(channel); 106 | usleep(1000); 107 | while (channel_results_.Count(channel) > 0) { 108 | usleep(200); 109 | } 110 | 111 | CHECK(channel_results_.Count(channel) == 0); 112 | usleep(1000); 113 | } 114 | 115 | 116 | void ConnectionMultiplexer::Run() { 117 | MessageProto message; 118 | zmq::message_t msg; 119 | string channel; 120 | bool got_request = false; 121 | 122 | while (!deconstructor_invoked_) { 123 | // Create new channel 124 | while (new_channel_queue_->Pop(&channel) == true) { 125 | if (channel_results_.Count(channel) > 0) { 126 | // Channel name already in use. Report an error and set new_connection_ 127 | // (which NewConnection() will return) to NULL. 128 | LOG(ERROR) << "Attempt to create channel that already exists: "<< channel; 129 | return ; 130 | } 131 | 132 | AtomicQueue* channel_queue = new AtomicQueue(); 133 | 134 | //LOG(ERROR) << local_node_id_ << ":ConnectionMultiplexer::Run(), creat new channel--:"<::iterator i; 137 | for (i = undelivered_messages_[channel].begin(); i != undelivered_messages_[channel].end(); ++i) { 138 | channel_queue->Push(*i); 139 | //LOG(ERROR) << local_node_id_ << ":ConnectionMultiplexer::Run(), creat new channel get undelivered_messages, channel:"<Pop(&channel); 148 | if (got_request == true) { 149 | if (channel_results_.Count(channel) > 0) { 150 | delete channel_results_.Lookup(channel); 151 | channel_results_.Erase(channel); 152 | } 153 | } 154 | 155 | 156 | // Forward next message from a remote node (if any). 157 | got_request = remote_in_->recv(&msg, ZMQ_NOBLOCK); 158 | if (got_request == true) { 159 | message.ParseFromArray(msg.data(), msg.size()); 160 | 161 | if (channel_results_.Count(message.destination_channel()) > 0) { 162 | (channel_results_.Lookup(message.destination_channel()))->Push(message); 163 | //LOG(ERROR) << local_node_id_ << ":ConnectionMultiplexer::Run(), receive a meesage1, channel:"<Pop(&message); 172 | if (got_request == true) { 173 | 174 | if (message.destination_node() == local_node_id_) { 175 | // Message is addressed to a local channel. If channel is valid, send the 176 | // message on, else store it to be delivered if the channel is ever created. 177 | if (channel_results_.Count(message.destination_channel()) > 0) { 178 | channel_results_.Lookup(message.destination_channel())->Push(message); 179 | } else { 180 | undelivered_messages_[message.destination_channel()].push_back(message); 181 | } 182 | } else { 183 | // Prepare message. 184 | string* message_string = new string(); 185 | message.SerializeToString(message_string); 186 | zmq::message_t msg(reinterpret_cast( 187 | const_cast(message_string->data())), 188 | message_string->size(), 189 | DeleteString, 190 | message_string); 191 | 192 | remote_out_[message.destination_node()]->send(msg); 193 | } 194 | } 195 | 196 | // Handle link_unlink request 197 | got_request = link_unlink_queue_->Pop(&message); 198 | if (got_request == true) { 199 | if (message.type() == MessageProto::LINK_CHANNEL) { 200 | AtomicQueue* main_queue = channel_results_.Lookup(message.main_channel()); 201 | CHECK(main_queue != NULL); 202 | channel_results_.Put(message.channel_request(), main_queue); 203 | // Forward on any messages sent to this channel before it existed. 204 | vector::iterator i; 205 | for (i = undelivered_messages_[message.channel_request()].begin(); 206 | i != undelivered_messages_[message.channel_request()].end(); 207 | ++i) { 208 | main_queue->Push(*i); 209 | } 210 | undelivered_messages_.erase(message.channel_request()); 211 | } else if (message.type() == MessageProto::UNLINK_CHANNEL) { 212 | //WriteLock l(&mutex_); 213 | channel_results_.Erase(message.channel_request()); 214 | } 215 | message.Clear(); 216 | } 217 | } 218 | } 219 | 220 | // Function to call multiplexer->Run() in a new pthread. 221 | void* ConnectionMultiplexer::RunMultiplexer(void *multiplexer) { 222 | reinterpret_cast(multiplexer)->Run(); 223 | return NULL; 224 | } 225 | 226 | void ConnectionMultiplexer::Send(const MessageProto& message) { 227 | MessageProto m; 228 | m.CopyFrom(message); 229 | send_message_queue_->Push(m); 230 | // send_message_queue_->Push(message); 231 | } 232 | 233 | 234 | void ConnectionMultiplexer::LinkChannel(const string& channel, const string& main_channel) { 235 | MessageProto m; 236 | m.set_type(MessageProto::LINK_CHANNEL); 237 | m.set_channel_request(channel); 238 | m.set_main_channel(main_channel); 239 | link_unlink_queue_->Push(m); 240 | } 241 | 242 | void ConnectionMultiplexer::UnlinkChannel(const string& channel) { 243 | MessageProto m; 244 | m.set_type(MessageProto::UNLINK_CHANNEL); 245 | m.set_channel_request(channel); 246 | link_unlink_queue_->Push(m); 247 | } 248 | 249 | -------------------------------------------------------------------------------- /src/machine/connection.h: -------------------------------------------------------------------------------- 1 | // Author: Kun Ren 2 | // 3 | // Library for handling messaging between system nodes. Each node generally owns 4 | // a ConnectionMultiplexer object as well as a Configuration object. 5 | 6 | #ifndef _DB_MACHINE_CONNECTION_H_ 7 | #define _DB_MACHINE_CONNECTION_H_ 8 | 9 | #include 10 | 11 | #include 12 | #include 13 | #include 14 | #include 15 | #include 16 | #include 17 | #include 18 | 19 | #include "machine/cluster_config.h" 20 | #include "common/utils.h" 21 | #include "common/mutex.h" 22 | #include "machine/zmq.hpp" 23 | #include "proto/message.pb.h" 24 | 25 | using std::map; 26 | using std::set; 27 | using std::string; 28 | using std::vector; 29 | using std::tr1::unordered_map; 30 | 31 | class ClusterConfig; 32 | 33 | class ConnectionMultiplexer { 34 | public: 35 | // Create a ConnectionMultiplexer that establishes two-way communication with 36 | // Connections for every other node specified by '*config' to exist. 37 | explicit ConnectionMultiplexer(ClusterConfig* config); 38 | 39 | ~ConnectionMultiplexer(); 40 | 41 | // Creates and registers a new channel with channel name 'channel', unless 42 | // the channel name is already in use, in which case NULL is returned. The 43 | // caller (not the multiplexer) owns of the newly created Connection object. 44 | 45 | void NewChannel(const string& channel); 46 | 47 | void DeleteChannel(const string& channel); 48 | 49 | void LinkChannel(const string& channel, const string& main_channel); 50 | 51 | void UnlinkChannel(const string& channel); 52 | 53 | bool GotMessage(const string& channel, MessageProto* message); 54 | 55 | void Send(const MessageProto& message); 56 | 57 | zmq::context_t* context() { return &context_; } 58 | 59 | uint64 Local_node_id() {return local_node_id_;} 60 | 61 | 62 | private: 63 | 64 | // Runs the Multiplexer's main loop. Run() is called in a new thread by the 65 | // constructor. 66 | void Run(); 67 | 68 | // Function to call multiplexer->Run() in a new pthread. 69 | static void* RunMultiplexer(void *multiplexer); 70 | 71 | // Separate pthread context in which to run the multiplexer's main loop. 72 | pthread_t thread_; 73 | 74 | // Pointer to Configuration instance used to construct this Multiplexer. 75 | // (Currently used primarily for finding 'this_node_id'.) 76 | ClusterConfig* configuration_; 77 | 78 | // Context shared by all Connection objects with channels to this 79 | // multiplexer. 80 | zmq::context_t context_; 81 | 82 | // Port on which to listen for incoming messages from other nodes. 83 | int port_; 84 | 85 | // Socket listening for messages from other nodes. Type = ZMQ_PULL. 86 | zmq::socket_t* remote_in_; 87 | 88 | // Sockets for outgoing traffic to other nodes. Keyed by node_id. 89 | // Type = ZMQ_PUSH. 90 | unordered_map remote_out_; 91 | 92 | AtomicMap*> channel_results_; 93 | 94 | AtomicQueue* link_unlink_queue_; 95 | 96 | AtomicQueue* new_channel_queue_; 97 | 98 | AtomicQueue* delete_channel_queue_; 99 | 100 | AtomicQueue* send_message_queue_; 101 | 102 | // Stores messages addressed to local channels that do not exist at the time 103 | // the message is received (so that they may be delivered if a connection is 104 | // ever created with the specified channel name). 105 | // 106 | unordered_map > undelivered_messages_; 107 | 108 | // False until the deconstructor is called. As soon as it is set to true, the 109 | // main loop sees it and stops. 110 | bool deconstructor_invoked_; 111 | 112 | uint64 local_node_id_; 113 | 114 | MutexRW mutex_; 115 | 116 | // DISALLOW_COPY_AND_ASSIGN 117 | ConnectionMultiplexer(const ConnectionMultiplexer&); 118 | ConnectionMultiplexer& operator=(const ConnectionMultiplexer&); 119 | }; 120 | 121 | 122 | 123 | #endif // _DB_MACHINE_CONNECTION_H_ 124 | 125 | -------------------------------------------------------------------------------- /src/machine/lowlatency_sequencer.h: -------------------------------------------------------------------------------- 1 | // Author: Kun Ren 2 | // 3 | 4 | 5 | #ifndef _DB_MACHINE_LOWLATENCY_SEQUENCER_H_ 6 | #define _DB_MACHINE_LOWLATENCY_SEQUENCER_H_ 7 | 8 | #include "machine/connection.h" 9 | #include "proto/message.pb.h" 10 | #include "log/local_paxos.h" 11 | #include "machine/client.h" 12 | #include "backend/simple_storage.h" 13 | 14 | class LowlatencySequencer { 15 | public: 16 | // The constructor creates background threads and starts the Sequencer's main 17 | // loops running. 18 | LowlatencySequencer(ClusterConfig* conf, ConnectionMultiplexer* connection, Client* client, LocalPaxos* paxos, Storage* storage, uint32 max_batch_size); 19 | 20 | // Halts the main loops. 21 | ~LowlatencySequencer(); 22 | 23 | private: 24 | // LowlatencySequencer's main loops: 25 | // 26 | // RunWriter: 27 | // while true: 28 | // Spend epoch_duration collecting client txn requests into a batch. 29 | // 30 | // RunReader: 31 | // while true: 32 | // Distribute the txns to relevant machines; 33 | // Send txns to other replicas; 34 | // Append the batch id to paxos log 35 | // 36 | // Executes in a background thread created and started by the constructor. 37 | void RunWriter(); 38 | void RunReader(); 39 | 40 | // Functions to start the Multiplexor's main loops, called in new pthreads by 41 | // the LowlatencySequencer's constructor. 42 | static void* RunSequencerWriter(void *arg); 43 | static void* RunSequencerReader(void *arg); 44 | 45 | // Sets '*nodes' to contain the node_id of every node participating in 'txn'. 46 | void FindParticipatingNodes(const TxnProto& txn, set* nodes); 47 | 48 | // Length of time spent collecting client requests before they are ordered, 49 | // batched, and sent out to schedulers. 50 | double epoch_duration_; 51 | 52 | // Configuration specifying node & system settings. 53 | ClusterConfig* configuration_; 54 | 55 | // Connection for sending and receiving protocol messages. 56 | ConnectionMultiplexer* connection_; 57 | 58 | // Client from which to get incoming txns. 59 | Client* client_; 60 | 61 | // Separate pthread contexts in which to run the sequencer's main loops. 62 | pthread_t writer_thread_; 63 | pthread_t reader_thread_; 64 | 65 | // False until the deconstructor is called. As soon as it is set to true, the 66 | // main loop sees it and stops. 67 | bool deconstructor_invoked_; 68 | 69 | LocalPaxos* paxos_log_; 70 | 71 | Storage* storage_; 72 | 73 | uint32 max_batch_size_; 74 | 75 | bool start_working_; 76 | 77 | 78 | }; 79 | #endif // _DB_MACHINE_LOWLATENCY_SEQUENCER_H_ 80 | -------------------------------------------------------------------------------- /src/machine/sequencer.cc: -------------------------------------------------------------------------------- 1 | // Author: Kun Ren 2 | // 3 | // 4 | // The sequencer component of the system is responsible for choosing a global 5 | // serial order of transactions to which execution must maintain equivalence. 6 | // 7 | 8 | #include "machine/sequencer.h" 9 | 10 | #ifdef LATENCY_TEST 11 | map sequencer_recv; 12 | map scheduler_unlock; 13 | std::atomic latency_counter; 14 | vector measured_latency; 15 | #endif 16 | 17 | void* Sequencer::RunSequencerWriter(void *arg) { 18 | reinterpret_cast(arg)->RunWriter(); 19 | return NULL; 20 | } 21 | 22 | void* Sequencer::RunSequencerReader(void *arg) { 23 | reinterpret_cast(arg)->RunReader(); 24 | return NULL; 25 | } 26 | 27 | Sequencer::Sequencer(ClusterConfig* conf, ConnectionMultiplexer* connection, Client* client, Paxos* paxos, uint32 max_batch_size) 28 | : epoch_duration_(0.005), configuration_(conf), connection_(connection), 29 | client_(client), deconstructor_invoked_(false), paxos_log_(paxos), max_batch_size_(max_batch_size) { 30 | // Start Sequencer main loops running in background thread. 31 | 32 | 33 | connection_->NewChannel("sequencer_"); 34 | 35 | start_working_ = false; 36 | 37 | cpu_set_t cpuset; 38 | pthread_attr_t attr_writer; 39 | pthread_attr_init(&attr_writer); 40 | CPU_ZERO(&cpuset); 41 | CPU_SET(2, &cpuset); 42 | // CPU_SET(6, &cpuset); 43 | pthread_attr_setaffinity_np(&attr_writer, sizeof(cpu_set_t), &cpuset); 44 | 45 | pthread_create(&writer_thread_, &attr_writer, RunSequencerWriter, reinterpret_cast(this)); 46 | 47 | CPU_ZERO(&cpuset); 48 | // CPU_SET(2, &cpuset); 49 | CPU_SET(6, &cpuset); 50 | pthread_attr_t attr_reader; 51 | pthread_attr_init(&attr_reader); 52 | pthread_attr_setaffinity_np(&attr_reader, sizeof(cpu_set_t), &cpuset); 53 | 54 | pthread_create(&reader_thread_, &attr_reader, RunSequencerReader, reinterpret_cast(this)); 55 | } 56 | 57 | Sequencer::~Sequencer() { 58 | deconstructor_invoked_ = true; 59 | pthread_join(writer_thread_, NULL); 60 | pthread_join(reader_thread_, NULL); 61 | } 62 | 63 | 64 | void Sequencer::RunWriter() { 65 | 66 | // Set up batch messages for each system node. 67 | MessageProto batch_message; 68 | batch_message.set_destination_channel("sequencer_"); 69 | batch_message.set_type(MessageProto::TXN_BATCH); 70 | uint64 batch_number; 71 | uint32 txn_id_offset; 72 | 73 | uint32 local_replica = configuration_->local_replica_id(); 74 | 75 | /**if (configuration_->local_node_id() == 2 || configuration_->local_node_id() == 3) { 76 | epoch_duration_ = 0.01; 77 | } else { 78 | epoch_duration_ = 100; 79 | }**/ 80 | 81 | #ifdef LATENCY_TEST 82 | latency_counter = 0; 83 | uint64 local_machine = configuration_->local_node_id(); 84 | #endif 85 | 86 | connection_->NewChannel("synchronization_sequencer_channel"); 87 | MessageProto synchronization_message; 88 | synchronization_message.set_type(MessageProto::EMPTY); 89 | synchronization_message.set_destination_channel("synchronization_sequencer_channel"); 90 | for (uint64 i = 0; i < (uint64)(configuration_->all_nodes_size()); i++) { 91 | synchronization_message.set_destination_node(i); 92 | if (i != static_cast(configuration_->local_node_id())) { 93 | connection_->Send(synchronization_message); 94 | } 95 | } 96 | 97 | uint32 synchronization_counter = 1; 98 | while (synchronization_counter < (uint64)(configuration_->all_nodes_size())) { 99 | synchronization_message.Clear(); 100 | if (connection_->GotMessage("synchronization_sequencer_channel", &synchronization_message)) { 101 | CHECK(synchronization_message.type() == MessageProto::EMPTY); 102 | synchronization_counter++; 103 | } 104 | } 105 | 106 | connection_->DeleteChannel("synchronization_sequencer_channel"); 107 | LOG(ERROR) << "In sequencer: After synchronization. Starting sequencer writer."; 108 | start_working_ = true; 109 | 110 | while (!deconstructor_invoked_) { 111 | // Begin epoch. 112 | batch_number = configuration_->GetGUID(); 113 | double epoch_start = GetTime(); 114 | batch_message.set_batch_number(batch_number); 115 | batch_message.clear_data(); 116 | 117 | // Collect txn requests for this epoch. 118 | txn_id_offset = 0; 119 | while (!deconstructor_invoked_ && 120 | GetTime() < epoch_start + epoch_duration_) { 121 | // Add next txn request to batch. 122 | if ((uint32)(batch_message.data_size()) < max_batch_size_) { 123 | TxnProto* txn; 124 | string txn_string; 125 | client_->GetTxn(&txn, batch_number * max_batch_size_ + txn_id_offset); 126 | 127 | txn->set_origin_replica(local_replica); 128 | txn->add_involved_replicas(local_replica); 129 | 130 | #ifdef LATENCY_TEST 131 | if (txn->txn_id() % SAMPLE_RATE == 0 && latency_counter < SAMPLES) { 132 | sequencer_recv[txn->txn_id()] = GetTime(); 133 | txn->set_generated_machine(local_machine); 134 | } 135 | #endif 136 | 137 | txn->SerializeToString(&txn_string); 138 | batch_message.add_data(txn_string); 139 | txn_id_offset++; 140 | 141 | delete txn; 142 | } else { 143 | usleep(50); 144 | } 145 | } 146 | 147 | 148 | // Send this epoch's transactions to the central machine of each replica 149 | for (uint32 i = 0; i < configuration_->replicas_size(); i++) { 150 | uint64 machine_id = configuration_->LookupMachineID(configuration_->HashBatchID(batch_message.batch_number()), i); 151 | //LOG(ERROR) << configuration_->local_node_id()<<": In sequencer reader: will send TXN_BATCH to :"<Send(batch_message); 154 | //LOG(ERROR) << configuration_->local_node_id()<<": In sequencer reader: after send TXN_BATCH to :"< batches; 165 | 166 | uint32 local_replica = configuration_->local_replica_id(); 167 | for (uint64 i = 0; i < configuration_->nodes_per_replica();i++) { 168 | batches[i].set_destination_channel("scheduler_"); 169 | batches[i].set_destination_node(configuration_->LookupMachineID(i, local_replica)); 170 | batches[i].set_type(MessageProto::TXN_SUBBATCH); 171 | } 172 | 173 | MessageProto message; 174 | uint64 batch_number; 175 | 176 | while (start_working_ != true) { 177 | usleep(100); 178 | } 179 | 180 | while (!deconstructor_invoked_) { 181 | 182 | bool got_message = connection_->GotMessage("sequencer_", &message); 183 | if (got_message == true) { 184 | if (message.type() == MessageProto::TXN_BATCH) { 185 | //LOG(ERROR) << configuration_->local_node_id()<< ":In sequencer reader: recevie TXN_BATCH message:"< readers; 194 | set writers; 195 | for (uint32 i = 0; i < (uint32)(txn.read_set_size()); i++) { 196 | KeyEntry key_entry = txn.read_set(i); 197 | uint64 mds = configuration_->LookupPartition(key_entry.key()); 198 | readers.insert(mds); 199 | } 200 | for (uint32 i = 0; i < (uint32)(txn.read_write_set_size()); i++) { 201 | KeyEntry key_entry = txn.read_write_set(i); 202 | uint64 mds = configuration_->LookupPartition(key_entry.key()); 203 | writers.insert(mds); 204 | readers.insert(mds); 205 | } 206 | 207 | for (set::iterator it = readers.begin(); it != readers.end(); ++it) { 208 | txn.add_readers(*it); 209 | } 210 | for (set::iterator it = writers.begin(); it != writers.end(); ++it) { 211 | txn.add_writers(*it); 212 | } 213 | 214 | string txn_data; 215 | txn.SerializeToString(&txn_data); 216 | 217 | // Compute union of 'readers' and 'writers' (store in 'readers'). 218 | for (set::iterator it = writers.begin(); it != writers.end(); ++it) { 219 | readers.insert(*it); 220 | } 221 | 222 | // Insert txn into appropriate batches. 223 | for (set::iterator it = readers.begin(); it != readers.end(); ++it) { 224 | batches[*it].add_data(txn_data); 225 | } 226 | } 227 | 228 | // Send this epoch's requests to all schedulers. 229 | for (map::iterator it = batches.begin(); it != batches.end(); ++it) { 230 | it->second.set_batch_number(batch_number); 231 | connection_->Send(it->second); 232 | it->second.clear_data(); 233 | } 234 | 235 | // Send “vote” to the head machines; 236 | MessageProto vote_message; 237 | vote_message.set_destination_channel("sequencer_"); 238 | vote_message.set_destination_node(0); 239 | vote_message.set_type(MessageProto::BATCH_VOTE); 240 | vote_message.add_misc_int(message.batch_number()); 241 | connection_->Send(vote_message); 242 | 243 | } else if (message.type() == MessageProto::BATCH_VOTE) { 244 | //LOG(ERROR) << configuration_->local_node_id()<< ":In sequencer reader: recevie BATCH_VOTE message:"<replicas_size()) { 258 | batch_votes_.erase(batch_id); 259 | } 260 | 261 | // If block is now written to (exactly) a majority of replicas, submit 262 | // to paxos leader. 263 | if (votes == configuration_->replicas_size() / 2 + 1) { 264 | //LOG(ERROR) << configuration_->local_node_id()<< ":In sequencer reader: recevie BATCH_VOTE message, will append:"<Append(batch_id); 266 | } 267 | } 268 | } 269 | 270 | if (got_message == false) { 271 | usleep(50); 272 | } 273 | } 274 | } 275 | -------------------------------------------------------------------------------- /src/machine/sequencer.h: -------------------------------------------------------------------------------- 1 | // Author: Kun Ren 2 | // 3 | 4 | 5 | #ifndef _DB_MACHINE_SEQUENCER_H_ 6 | #define _DB_MACHINE_SEQUENCER_H_ 7 | 8 | #include "machine/connection.h" 9 | #include "proto/message.pb.h" 10 | #include "log/paxos.h" 11 | #include "machine/client.h" 12 | 13 | class Sequencer { 14 | public: 15 | // The constructor creates background threads and starts the Sequencer's main 16 | // loops running. 17 | Sequencer(ClusterConfig* conf, ConnectionMultiplexer* connection, Client* client, Paxos* paxos, uint32 max_batch_size); 18 | 19 | // Halts the main loops. 20 | ~Sequencer(); 21 | 22 | private: 23 | // Sequencer's main loops: 24 | // 25 | // RunWriter: 26 | // while true: 27 | // Spend epoch_duration collecting client txn requests into a batch. 28 | // 29 | // RunReader: 30 | // while true: 31 | // Distribute the txns to relevant machines; 32 | // Send txns to other replicas; 33 | // Append the batch id to paxos log 34 | // 35 | // Executes in a background thread created and started by the constructor. 36 | void RunWriter(); 37 | void RunReader(); 38 | 39 | // Functions to start the Multiplexor's main loops, called in new pthreads by 40 | // the Sequencer's constructor. 41 | static void* RunSequencerWriter(void *arg); 42 | static void* RunSequencerReader(void *arg); 43 | 44 | // Sets '*nodes' to contain the node_id of every node participating in 'txn'. 45 | void FindParticipatingNodes(const TxnProto& txn, set* nodes); 46 | 47 | // Length of time spent collecting client requests before they are ordered, 48 | // batched, and sent out to schedulers. 49 | double epoch_duration_; 50 | 51 | // Configuration specifying node & system settings. 52 | ClusterConfig* configuration_; 53 | 54 | // Connection for sending and receiving protocol messages. 55 | ConnectionMultiplexer* connection_; 56 | 57 | // Client from which to get incoming txns. 58 | Client* client_; 59 | 60 | // Separate pthread contexts in which to run the sequencer's main loops. 61 | pthread_t writer_thread_; 62 | pthread_t reader_thread_; 63 | 64 | // False until the deconstructor is called. As soon as it is set to true, the 65 | // main loop sees it and stops. 66 | bool deconstructor_invoked_; 67 | 68 | Paxos* paxos_log_; 69 | 70 | uint32 max_batch_size_; 71 | 72 | // Number of votes for each batch (used only by machine 0). 73 | map batch_votes_; 74 | 75 | bool start_working_; 76 | }; 77 | #endif // _DB_MACHINE_SEQUENCER_H_ 78 | -------------------------------------------------------------------------------- /src/machine/zmq.hpp: -------------------------------------------------------------------------------- 1 | /* 2 | Copyright (c) 2007-2011 iMatix Corporation 3 | Copyright (c) 2007-2011 Other contributors as noted in the AUTHORS file 4 | 5 | This file is part of 0MQ. 6 | 7 | 0MQ is free software; you can redistribute it and/or modify it under 8 | the terms of the GNU Lesser General Public License as published by 9 | the Free Software Foundation; either version 3 of the License, or 10 | (at your option) any later version. 11 | 12 | 0MQ is distributed in the hope that it will be useful, 13 | but WITHOUT ANY WARRANTY; without even the implied warranty of 14 | MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the 15 | GNU Lesser General Public License for more details. 16 | 17 | You should have received a copy of the GNU Lesser General Public License 18 | along with this program. If not, see . 19 | */ 20 | // Modified: Kun Ren 21 | 22 | #ifndef __ZMQ_HPP_INCLUDED__ 23 | #define __ZMQ_HPP_INCLUDED__ 24 | 25 | #include "zmq.h" 26 | 27 | #include 28 | #include 29 | #include 30 | 31 | // Detect whether the compiler supports C++11 rvalue references. 32 | #if (defined(__GNUC__) && (__GNUC__ > 4 || \ 33 | (__GNUC__ == 4 && __GNUC_MINOR__ > 2)) && \ 34 | defined(__GXX_EXPERIMENTAL_CXX0X__)) 35 | #define ZMQ_HAS_RVALUE_REFS 36 | #define ZMQ_DELETED_FUNCTION = delete 37 | #elif defined(__clang__) 38 | #if __has_feature(cxx_rvalue_references) 39 | #define ZMQ_HAS_RVALUE_REFS 40 | #endif 41 | 42 | #if __has_feature(cxx_deleted_functions) 43 | #define ZMQ_DELETED_FUNCTION = delete 44 | #endif 45 | #elif defined(_MSC_VER) && (_MSC_VER >= 1600) 46 | #define ZMQ_HAS_RVALUE_REFS 47 | #define ZMQ_DELETED_FUNCTION 48 | #else 49 | #define ZMQ_DELETED_FUNCTION 50 | #endif 51 | 52 | // In order to prevent unused variable warnings when building in non-debug 53 | // mode use this macro to make assertions. 54 | #ifndef NDEBUG 55 | # define ZMQ_ASSERT(expression) assert(expression) 56 | #else 57 | # define ZMQ_ASSERT(expression) (expression) 58 | #endif 59 | 60 | 61 | namespace zmq 62 | { 63 | 64 | typedef zmq_free_fn free_fn; 65 | typedef zmq_pollitem_t pollitem_t; 66 | 67 | class error_t : public std::exception 68 | { 69 | public: 70 | 71 | error_t () : errnum (zmq_errno ()) {} 72 | 73 | virtual const char *what () const throw () 74 | { 75 | return zmq_strerror (errnum); 76 | } 77 | 78 | int num () const 79 | { 80 | return errnum; 81 | } 82 | 83 | private: 84 | 85 | int errnum; 86 | }; 87 | 88 | inline int poll (zmq_pollitem_t *items_, int nitems_, long timeout_ = -1) 89 | { 90 | int rc = zmq_poll (items_, nitems_, timeout_); 91 | return rc; 92 | } 93 | 94 | inline void version (int *major_, int *minor_, int *patch_) 95 | { 96 | zmq_version (major_, minor_, patch_); 97 | } 98 | 99 | class message_t 100 | { 101 | friend class socket_t; 102 | 103 | public: 104 | 105 | inline message_t () 106 | { 107 | zmq_msg_init (&msg); 108 | } 109 | 110 | inline explicit message_t (size_t size_) 111 | { 112 | zmq_msg_init_size (&msg, size_); 113 | } 114 | 115 | inline message_t (void *data_, size_t size_, free_fn *ffn_, 116 | void *hint_ = NULL) 117 | { 118 | zmq_msg_init_data (&msg, data_, size_, ffn_, hint_); 119 | } 120 | 121 | #ifdef ZMQ_HAS_RVALUE_REFS 122 | inline message_t (message_t &&rhs) : msg (rhs.msg) 123 | { 124 | zmq_msg_init (&rhs.msg); 125 | } 126 | 127 | inline message_t &operator = (message_t &&rhs) 128 | { 129 | std::swap (msg, rhs.msg); 130 | return *this; 131 | } 132 | #endif 133 | 134 | inline ~message_t () 135 | { 136 | int rc = zmq_msg_close (&msg); 137 | ZMQ_ASSERT (rc == 0); 138 | } 139 | 140 | inline void rebuild () 141 | { 142 | zmq_msg_close (&msg); 143 | zmq_msg_init (&msg); 144 | } 145 | 146 | inline void rebuild (size_t size_) 147 | { 148 | zmq_msg_close (&msg); 149 | zmq_msg_init_size (&msg, size_); 150 | } 151 | 152 | inline void rebuild (void *data_, size_t size_, free_fn *ffn_, 153 | void *hint_ = NULL) 154 | { 155 | zmq_msg_close (&msg); 156 | zmq_msg_init_data (&msg, data_, size_, ffn_, hint_); 157 | } 158 | 159 | inline void move (message_t *msg_) 160 | { 161 | zmq_msg_move (&msg, &(msg_->msg)); 162 | } 163 | 164 | inline void copy (message_t *msg_) 165 | { 166 | zmq_msg_copy (&msg, &(msg_->msg)); 167 | } 168 | 169 | inline void *data () 170 | { 171 | return zmq_msg_data (&msg); 172 | } 173 | 174 | inline const void* data () const 175 | { 176 | return zmq_msg_data (const_cast(&msg)); 177 | } 178 | 179 | inline size_t size () const 180 | { 181 | return zmq_msg_size (const_cast(&msg)); 182 | } 183 | 184 | private: 185 | 186 | // The underlying message 187 | zmq_msg_t msg; 188 | 189 | // Disable implicit message copying, so that users won't use shared 190 | // messages (less efficient) without being aware of the fact. 191 | message_t (const message_t&); 192 | void operator = (const message_t&); 193 | }; 194 | 195 | class context_t 196 | { 197 | friend class socket_t; 198 | 199 | public: 200 | 201 | inline explicit context_t (int io_threads_) 202 | { 203 | ptr = zmq_init (io_threads_); 204 | } 205 | 206 | #ifdef ZMQ_HAS_RVALUE_REFS 207 | inline context_t (context_t &&rhs) : ptr (rhs.ptr) 208 | { 209 | rhs.ptr = NULL; 210 | } 211 | inline context_t &operator = (context_t &&rhs) 212 | { 213 | std::swap (ptr, rhs.ptr); 214 | return *this; 215 | } 216 | #endif 217 | 218 | inline ~context_t () 219 | { 220 | close(); 221 | } 222 | 223 | inline void close() 224 | { 225 | if (ptr == NULL) 226 | return; 227 | int rc = zmq_term (ptr); 228 | ZMQ_ASSERT (rc == 0); 229 | ptr = NULL; 230 | } 231 | 232 | // Be careful with this, it's probably only useful for 233 | // using the C api together with an existing C++ api. 234 | // Normally you should never need to use this. 235 | inline operator void* () 236 | { 237 | return ptr; 238 | } 239 | 240 | private: 241 | 242 | void *ptr; 243 | 244 | context_t (const context_t&); 245 | void operator = (const context_t&); 246 | }; 247 | 248 | class socket_t 249 | { 250 | public: 251 | 252 | inline socket_t (context_t &context_, int type_) 253 | { 254 | ptr = zmq_socket (context_.ptr, type_); 255 | } 256 | 257 | #ifdef ZMQ_HAS_RVALUE_REFS 258 | inline socket_t(socket_t&& rhs) : ptr(rhs.ptr) 259 | { 260 | rhs.ptr = NULL; 261 | } 262 | inline socket_t& operator=(socket_t&& rhs) 263 | { 264 | std::swap(ptr, rhs.ptr); 265 | return *this; 266 | } 267 | #endif 268 | 269 | inline ~socket_t () 270 | { 271 | close(); 272 | } 273 | 274 | inline operator void* () 275 | { 276 | return ptr; 277 | } 278 | 279 | inline void close() 280 | { 281 | if(ptr == NULL) 282 | // already closed 283 | return ; 284 | int rc = zmq_close (ptr); 285 | ZMQ_ASSERT (rc == 0); 286 | ptr = 0 ; 287 | } 288 | 289 | inline void setsockopt (int option_, const void *optval_, 290 | size_t optvallen_) 291 | { 292 | zmq_setsockopt (ptr, option_, optval_, optvallen_); 293 | } 294 | 295 | inline void getsockopt (int option_, void *optval_, 296 | size_t *optvallen_) 297 | { 298 | zmq_getsockopt (ptr, option_, optval_, optvallen_); 299 | } 300 | 301 | inline void bind (const char *addr_) 302 | { 303 | zmq_bind (ptr, addr_); 304 | } 305 | 306 | inline void connect (const char *addr_) 307 | { 308 | zmq_connect (ptr, addr_); 309 | } 310 | 311 | inline bool connected() 312 | { 313 | return(ptr != NULL); 314 | } 315 | 316 | inline size_t send (const void *buf_, size_t len_, int flags_ = 0) 317 | { 318 | int nbytes = zmq_send (ptr, buf_, len_, flags_); 319 | if (nbytes >= 0) 320 | return (size_t) nbytes; 321 | if (zmq_errno () == EAGAIN) 322 | return 0; 323 | } 324 | 325 | inline bool send (message_t &msg_, int flags_ = 0) 326 | { 327 | int nbytes = zmq_msg_send (&(msg_.msg), ptr, flags_); 328 | if (nbytes >= 0) 329 | return true; 330 | if (zmq_errno () == EAGAIN) 331 | return false; 332 | exit(1); 333 | } 334 | 335 | inline size_t recv (void *buf_, size_t len_, int flags_ = 0) 336 | { 337 | int nbytes = zmq_recv (ptr, buf_, len_, flags_); 338 | if (nbytes >= 0) 339 | return (size_t) nbytes; 340 | if (zmq_errno () == EAGAIN) 341 | return 0; 342 | exit(1); 343 | } 344 | 345 | inline bool recv (message_t *msg_, int flags_ = 0) 346 | { 347 | int nbytes = zmq_msg_recv (&(msg_->msg), ptr, flags_); 348 | if (nbytes >= 0) 349 | return true; 350 | if (zmq_errno () == EAGAIN) 351 | return false; 352 | exit(1); 353 | } 354 | 355 | private: 356 | 357 | void *ptr; 358 | 359 | socket_t (const socket_t&) ZMQ_DELETED_FUNCTION; 360 | void operator = (const socket_t&) ZMQ_DELETED_FUNCTION; 361 | }; 362 | 363 | } 364 | 365 | #endif 366 | 367 | -------------------------------------------------------------------------------- /src/proto/Makefile.inc: -------------------------------------------------------------------------------- 1 | # Create template specific variables 2 | UC_DIR := PROTO 3 | LC_DIR := proto 4 | 5 | SRCS := 6 | EXES := 7 | TEST := 8 | PROTOS := proto/scalar.proto $(wildcard proto/*.proto) 9 | DEPS := 10 | 11 | ################################################################################ 12 | 13 | # Link the template to avoid redundancy 14 | include $(MAKEFILE_TEMPLATE) 15 | 16 | 17 | -------------------------------------------------------------------------------- /src/proto/cluster_config.proto: -------------------------------------------------------------------------------- 1 | // Author: Kun Ren 2 | // 3 | 4 | // ClusterConfig is essentially a collection of MachineInfo entries, each 5 | // of which stores the MachineID, host (ip address), and listen port for a 6 | // machine in the cluster. 7 | syntax = "proto2"; 8 | message MachineInfo { 9 | optional uint64 id = 1; 10 | optional uint32 replica = 2; 11 | optional string host = 3; 12 | optional int32 port = 4; 13 | }; 14 | 15 | // A ClusterConfig represents a description of a collection of Machines. 16 | // ClusterConfigProto is a protobuf version of that. 17 | message ClusterConfigProto { 18 | repeated MachineInfo machines = 1; 19 | }; 20 | 21 | -------------------------------------------------------------------------------- /src/proto/message.proto: -------------------------------------------------------------------------------- 1 | // Author: Kun Ren 2 | // 3 | // Protocol buffer used for all network messages in the system. 4 | syntax = "proto2"; 5 | message MessageProto { 6 | // Node to which this message should be sent. 7 | required uint64 destination_node = 1; 8 | 9 | // Channel to which this message shall be delivered when it arrives at node 10 | // 'destination_node'. 11 | required string destination_channel = 2; 12 | 13 | // Node from which the message originated. 14 | optional uint64 source_node = 3; 15 | 16 | // Channel from which the message originated. 17 | optional int32 source_channel = 4; 18 | 19 | // Every type of network message should get an entry here. 20 | enum MessageType { 21 | EMPTY = 0; 22 | TXN_PROTO = 1; 23 | TXN_BATCH = 2; 24 | READ_RESULT = 3; 25 | LINK_CHANNEL = 4; // [Connection implementation specific.] 26 | UNLINK_CHANNEL = 5; // [Connection implementation specific.] 27 | TXN_PTR = 6; 28 | MESSAGE_PTR = 7; 29 | TXN_SUBBATCH = 8; 30 | PAXOS_BATCH_ORDER = 9; 31 | PAXOS_COMMIT = 10; 32 | PAXOS_DATA = 11; 33 | PAXOS_DATA_ACK = 12; 34 | BATCH_VOTE = 13; 35 | TXN_FORWORD = 14; 36 | BATCH_SUBMIT = 15; 37 | MR_TXNS_BATCH = 16; 38 | NEW_SEQUENCE = 17; 39 | NEW_SEQUENCE_ACK = 18; 40 | MASTER_LOOKUP = 19; 41 | MASTER_LOOKUP_RESULT = 20; 42 | SYNCHRONIZE = 21; 43 | SYNCHRONIZE_ACK = 22; 44 | }; 45 | required MessageType type = 9; 46 | 47 | // Actual data for the message being carried, to be deserialized into a 48 | // protocol message object of type depending on 'type'. In TXN_PROTO and 49 | // TXN_BATCH messages, 'data' contains are one and any number of TxnProtos, 50 | // respectively. 51 | repeated bytes data = 11; 52 | 53 | // Pointer to actual data for message being carried. Can only be used for 54 | // messages between threads. 55 | repeated int64 data_ptr = 12; 56 | 57 | // For TXN_BATCH messages, 'batch_number' identifies the epoch of the txn 58 | // batch being sent. 59 | optional uint64 batch_number = 21; 60 | 61 | // For READ_RESULT messages, 'keys(i)' and 'values(i)' store the key and 62 | // result of a read, respectively. 63 | repeated bytes keys = 31; 64 | repeated bytes values = 32; 65 | 66 | // Stick miscellaneous stuff in here if you really want to. 67 | repeated bool misc_bool = 41; 68 | repeated uint64 misc_int = 42; 69 | repeated string misc_string = 43; 70 | repeated double misc_double = 44; 71 | 72 | // For (UN)LINK_CHANNEL messages, specifies the main channel of the requesting 73 | // Connection object. 74 | optional string main_channel = 1001; 75 | 76 | // For (UN)LINK_CHANNEL messages, specifies the channel to be (un)linked 77 | // to the requesting Connection object. 78 | optional string channel_request = 1002; 79 | } 80 | 81 | -------------------------------------------------------------------------------- /src/proto/scalar.proto: -------------------------------------------------------------------------------- 1 | // Author: Kun Ren 2 | // 3 | // Encoding of some kind of numeric or string value. 4 | syntax = "proto2"; 5 | message Scalar { 6 | // Specifies actual type of scalar value. 7 | enum ScalarType { 8 | UNIT = 1; 9 | BOOL = 2; 10 | INT32 = 3; 11 | INT64 = 4; 12 | UINT32 = 5; 13 | UINT64 = 6; 14 | FLOAT = 7; 15 | DOUBLE = 8; 16 | STRING = 9; 17 | }; 18 | required ScalarType type = 1 [default = UNIT]; 19 | 20 | // Actual scalar value. Exactly one of these must exist, and it must 21 | // correspond to the ScalarType specified by 'type' (unless type == UNIT, 22 | // in which case no value is set). 23 | optional bool bool_value = 2; 24 | optional int32 int32_value = 3; 25 | optional int64 int64_value = 4; 26 | optional uint32 uint32_value = 5; 27 | optional uint64 uint64_value = 6; 28 | optional float float_value = 7; 29 | optional double double_value = 8; 30 | optional string string_value = 9; 31 | }; 32 | 33 | // Possibly useful for when a protobuf is needed but you don't actually have 34 | // anything you need to say. 35 | message EmptyProto {} 36 | 37 | message UInt64Pair { 38 | optional uint64 first = 1; 39 | optional uint64 second = 2; 40 | } 41 | 42 | message Sequence { 43 | repeated uint64 batch_ids = 1; 44 | } 45 | 46 | message SequenceBatch { 47 | repeated Sequence sequence_batch = 1; 48 | } 49 | 50 | -------------------------------------------------------------------------------- /src/proto/txn.proto: -------------------------------------------------------------------------------- 1 | // Author: Kun Ren 2 | // 3 | 4 | syntax = "proto2"; 5 | 6 | message KeyEntry { 7 | required bytes key = 1; 8 | required uint32 master = 2; 9 | required uint64 counter = 3; 10 | } 11 | 12 | message KeyEntries { 13 | repeated KeyEntry entries = 1; 14 | } 15 | 16 | message RemoteResultsEntry { 17 | required bytes key = 1; 18 | required bytes value = 2; 19 | required uint32 master = 3; 20 | required uint64 counter = 4; 21 | } 22 | 23 | message RemoteResultsEntries { 24 | repeated RemoteResultsEntry entries = 1; 25 | } 26 | 27 | message TxnProto { 28 | // Globally unique transaction id, specifying global order. 29 | required uint64 txn_id = 1; 30 | 31 | // Specifies which stored procedure to invoke at execution time. 32 | optional uint32 txn_type = 10; 33 | 34 | // True if transaction is known to span multiple database nodes. 35 | //optional bool multipartition = 12; 36 | 37 | // Keys of objects read (but not modified) by this transaction. 38 | repeated KeyEntry read_set = 20; 39 | 40 | // Keys of objects modified (but not read) by this transaction. 41 | repeated KeyEntry write_set = 21; 42 | 43 | // Keys of objects read AND modified by this transaction. 44 | repeated KeyEntry read_write_set = 22; 45 | 46 | // Arguments to be passed when invoking the stored procedure to execute this 47 | // transaction. 'arg' is a serialized protocol message. The client and backend 48 | // application code is assumed to know how to interpret this protocol message 49 | // based on 'txn_type'. 50 | optional bytes arg = 23; 51 | 52 | // Transaction status. 53 | enum Status { 54 | NEW = 0; 55 | ACTIVE = 1; 56 | COMMITTED = 2; 57 | ABORTED = 3; 58 | ABORTED_WITHOUT_LOCK = 4; 59 | }; 60 | 61 | optional Status status = 30 [default = NEW]; 62 | 63 | // Node ids of nodes that participate as readers and writers in this txn. 64 | repeated uint64 readers = 40; 65 | repeated uint64 writers = 41; 66 | 67 | //------- Below is for low latency CalvinDB 68 | optional uint32 origin_replica = 50 [default = 0]; 69 | // whether the action is the new_generated cross-replica action 70 | optional bool new_generated = 51 [default = false]; 71 | repeated uint32 involved_replicas = 52; 72 | optional bool fake_txn = 53 [default = false]; 73 | 74 | //------- Below is for remaster transactions 75 | optional bool remaster_txn = 54 [default = false]; 76 | optional uint32 client_replica = 55; 77 | optional uint32 remaster_from = 56 [default = 99999]; 78 | optional uint32 remaster_to = 57 [default = 99999]; 79 | optional bool wait_for_remaster_pros = 58 [default = false]; 80 | 81 | // Also should take some information about which records will be remaster 82 | // This is to measure latency 83 | optional uint64 generated_machine = 64 [default = 1000]; 84 | } 85 | 86 | message LookupMasterEntry { 87 | required uint64 txn_id = 1; 88 | repeated bytes keys = 2; 89 | } 90 | 91 | message LookupMasterResultEntry { 92 | required uint64 txn_id = 1; 93 | repeated KeyEntry key_entries = 2; 94 | } 95 | 96 | 97 | 98 | -------------------------------------------------------------------------------- /src/scheduler/Makefile.inc: -------------------------------------------------------------------------------- 1 | # Create template specific variables 2 | UC_DIR := SCHEDULER 3 | LC_DIR := scheduler 4 | 5 | SRCS := scheduler/deterministic_lock_manager.cc \ 6 | scheduler/deterministic_scheduler.cc 7 | 8 | EXES := 9 | DEPS := $(PROTO_OBJS) $(COMMON_OBJS) $(BACKEND_OBJS) $(APPLICATION_OBJS) $(MACHINE_OBJS) 10 | 11 | # Link the template to avoid redundancy 12 | include $(MAKEFILE_TEMPLATE) 13 | 14 | -------------------------------------------------------------------------------- /src/scheduler/deterministic_lock_manager.cc: -------------------------------------------------------------------------------- 1 | // Author: Kun Ren 2 | // 3 | // Lock manager implementing deterministic two-phase locking as described in 4 | // 'The Case for Determinism in Database Systems'. 5 | 6 | #include "scheduler/deterministic_lock_manager.h" 7 | 8 | #include 9 | 10 | #include "proto/txn.pb.h" 11 | 12 | using std::vector; 13 | 14 | DeterministicLockManager::DeterministicLockManager( 15 | AtomicQueue* ready_txns, 16 | ClusterConfig* config, uint32 mode) 17 | : ready_txns_(ready_txns), configuration_(config), mode_(mode) { 18 | for (int i = 0; i < TABLE_SIZE; i++) { 19 | lock_table_[i] = new deque(); 20 | } 21 | } 22 | 23 | int DeterministicLockManager::Lock(TxnProto* txn) { 24 | int not_acquired = 0; 25 | uint32 origin = txn->origin_replica(); 26 | 27 | // Handle read/write lock requests. 28 | for (int i = 0; i < txn->read_write_set_size(); i++) { 29 | KeyEntry key_entry = txn->read_write_set(i); 30 | if (mode_ != 0 && key_entry.master() != origin) { 31 | continue; 32 | } 33 | 34 | // Only lock local keys. 35 | if (IsLocal(key_entry.key())) { 36 | deque* key_requests = lock_table_[Hash(key_entry.key())]; 37 | 38 | deque::iterator it; 39 | for(it = key_requests->begin(); it != key_requests->end() && it->key != key_entry.key(); ++it) { 40 | } 41 | 42 | deque* requests; 43 | if (it == key_requests->end()) { 44 | requests = new deque(); 45 | key_requests->push_back(KeysList(key_entry.key(), requests)); 46 | } else { 47 | requests = it->locksrequest; 48 | } 49 | 50 | // Only need to request this if lock txn hasn't already requested it. 51 | if (requests->empty() || txn != requests->back().txn) { 52 | requests->push_back(LockRequest(WRITE, txn)); 53 | // Write lock request fails if there is any previous request at all. 54 | if (requests->size() > 1) 55 | not_acquired++; 56 | } 57 | } 58 | } 59 | 60 | // Handle read lock requests. This is last so that we don't have to deal with 61 | // upgrading lock requests from read to write on hash collisions. 62 | for (int i = 0; i < txn->read_set_size(); i++) { 63 | KeyEntry key_entry = txn->read_set(i); 64 | 65 | if (mode_ != 0 && key_entry.master() != origin) { 66 | continue; 67 | } 68 | 69 | // Only lock local keys. 70 | if (IsLocal(key_entry.key())) { 71 | deque* key_requests = lock_table_[Hash(key_entry.key())]; 72 | 73 | deque::iterator it; 74 | for(it = key_requests->begin(); it != key_requests->end() && it->key != key_entry.key(); ++it) { 75 | } 76 | 77 | deque* requests; 78 | if (it == key_requests->end()) { 79 | requests = new deque(); 80 | key_requests->push_back(KeysList(key_entry.key(), requests)); 81 | } else { 82 | requests = it->locksrequest; 83 | } 84 | 85 | // Only need to request this if lock txn hasn't already requested it. 86 | if (requests->empty() || txn != requests->back().txn) { 87 | requests->push_back(LockRequest(READ, txn)); 88 | // Read lock request fails if there is any previous write request. 89 | for (deque::iterator it = requests->begin(); 90 | it != requests->end(); ++it) { 91 | if (it->mode == WRITE) { 92 | not_acquired++; 93 | break; 94 | } 95 | } 96 | } 97 | } 98 | } 99 | 100 | // Record and return the number of locks that the txn is blocked on. 101 | if (not_acquired > 0) { 102 | txn_waits_[txn] = not_acquired; 103 | } else { 104 | ready_txns_->Push(txn); 105 | } 106 | return not_acquired; 107 | } 108 | 109 | void DeterministicLockManager::Release(TxnProto* txn) { 110 | uint32 origin = txn->origin_replica(); 111 | 112 | for (int i = 0; i < txn->read_set_size(); i++) { 113 | KeyEntry key_entry = txn->read_set(i); 114 | 115 | if (mode_ != 0 && key_entry.master() != origin) { 116 | continue; 117 | } 118 | 119 | if (IsLocal(key_entry.key())) { 120 | Release(key_entry.key(), txn); 121 | } 122 | } 123 | // Currently commented out because nothing in any write set can conflict 124 | // in TPCC or Microbenchmark. 125 | // for (int i = 0; i < txn->write_set_size(); i++) 126 | // if (IsLocal(txn->write_set(i))) 127 | // Release(txn->write_set(i), txn); 128 | for (int i = 0; i < txn->read_write_set_size(); i++) { 129 | KeyEntry key_entry = txn->read_write_set(i); 130 | 131 | if (mode_ != 0 && key_entry.master() != origin) { 132 | continue; 133 | } 134 | 135 | if (IsLocal(key_entry.key())) { 136 | //LOG(ERROR) <local_node_id()<<" :"<txn_id() <<" :$$$$$$ In Release: release key:"<* key_requests = lock_table_[Hash(key)]; 145 | 146 | deque::iterator it1; 147 | for(it1 = key_requests->begin(); 148 | it1 != key_requests->end() && it1->key != key; ++it1) { 149 | } 150 | deque* requests = it1->locksrequest; 151 | 152 | 153 | // Seek to the target request. Note whether any write lock requests precede 154 | // the target. 155 | bool write_requests_precede_target = false; 156 | deque::iterator it; 157 | for (it = requests->begin(); 158 | it != requests->end() && it->txn != txn; ++it) { 159 | if (it->mode == WRITE) 160 | write_requests_precede_target = true; 161 | } 162 | 163 | // If we found the request, erase it. No need to do anything otherwise. 164 | if (it != requests->end()) { 165 | // Save an iterator pointing to the target to call erase on after handling 166 | // lock inheritence, since erase(...) trashes all iterators. 167 | deque::iterator target = it; 168 | 169 | // If there are more requests following the target request, one or more 170 | // may need to be granted as a result of the target's release. 171 | ++it; 172 | if (it != requests->end()) { 173 | vector new_owners; 174 | // Grant subsequent request(s) if: 175 | // (a) The canceled request held a write lock. 176 | // (b) The canceled request held a read lock ALONE. 177 | // (c) The canceled request was a write request preceded only by read 178 | // requests and followed by one or more read requests. 179 | if (target == requests->begin() && 180 | (target->mode == WRITE || 181 | (target->mode == READ && it->mode == WRITE))) { // (a) or (b) 182 | // If a write lock request follows, grant it. 183 | if (it->mode == WRITE) 184 | new_owners.push_back(it->txn); 185 | // If a sequence of read lock requests follows, grant all of them. 186 | for (; it != requests->end() && it->mode == READ; ++it) 187 | new_owners.push_back(it->txn); 188 | } else if (!write_requests_precede_target && 189 | target->mode == WRITE && it->mode == READ) { // (c) 190 | // If a sequence of read lock requests follows, grant all of them. 191 | for (; it != requests->end() && it->mode == READ; ++it) 192 | new_owners.push_back(it->txn); 193 | } 194 | 195 | // Handle txns with newly granted requests that may now be ready to run. 196 | for (uint64 j = 0; j < new_owners.size(); j++) { 197 | txn_waits_[new_owners[j]]--; 198 | if (txn_waits_[new_owners[j]] == 0) { 199 | // The txn that just acquired the released lock is no longer waiting 200 | // on any lock requests. 201 | ready_txns_->Push(new_owners[j]); 202 | txn_waits_.erase(new_owners[j]); 203 | } 204 | } 205 | } 206 | 207 | // Now it is safe to actually erase the target request. 208 | requests->erase(target); 209 | if (requests->size() == 0) { 210 | delete requests; 211 | key_requests->erase(it1); 212 | } 213 | 214 | } 215 | } 216 | 217 | -------------------------------------------------------------------------------- /src/scheduler/deterministic_lock_manager.h: -------------------------------------------------------------------------------- 1 | // Author: Kun Ren 2 | // 3 | // Lock manager implementing deterministic two-phase locking as described in 4 | // 'The Case for Determinism in Database Systems'. 5 | 6 | #ifndef _DB_SCHEDULER_DETERMINISTIC_LOCK_MANAGER_H_ 7 | #define _DB_SCHEDULER_DETERMINISTIC_LOCK_MANAGER_H_ 8 | 9 | #include 10 | #include 11 | 12 | #include "machine/cluster_config.h" 13 | #include "scheduler/lock_manager.h" 14 | #include "common/utils.h" 15 | 16 | using std::tr1::unordered_map; 17 | using std::deque; 18 | 19 | #define TABLE_SIZE 1000000 20 | 21 | class TxnProto; 22 | 23 | class DeterministicLockManager { 24 | public: 25 | DeterministicLockManager(AtomicQueue* ready_txns, 26 | ClusterConfig* config, uint32 mode); 27 | virtual ~DeterministicLockManager() {} 28 | virtual int Lock(TxnProto* txn); 29 | virtual void Release(const Key& key, TxnProto* txn); 30 | virtual void Release(TxnProto* txn); 31 | 32 | private: 33 | int Hash(const Key& key) { 34 | uint64 hash = 2166136261; 35 | for (size_t i = 0; i < key.size(); i++) { 36 | hash = hash ^ (key[i]); 37 | hash = hash * 16777619; 38 | } 39 | return hash % TABLE_SIZE; 40 | } 41 | 42 | bool IsLocal(const Key& key) { 43 | return configuration_->LookupPartition(key) == configuration_->relative_node_id(); 44 | } 45 | 46 | // The DeterministicLockManager's lock table tracks all lock requests. For a 47 | // given key, if 'lock_table_' contains a nonempty queue, then the item with 48 | // that key is locked and either: 49 | // (a) first element in the queue specifies the owner if that item is a 50 | // request for a write lock, or 51 | // (b) a read lock is held by all elements of the longest prefix of the queue 52 | // containing only read lock requests. 53 | // Note: using STL deque rather than queue for erase(iterator position). 54 | struct LockRequest { 55 | LockRequest(LockMode m, TxnProto* t) : txn(t), mode(m) {} 56 | TxnProto* txn; // Pointer to txn requesting the lock. 57 | LockMode mode; // Specifies whether this is a read or write lock request. 58 | }; 59 | 60 | struct KeysList { 61 | KeysList(Key m, deque* t) : key(m), locksrequest(t) {} 62 | Key key; 63 | deque* locksrequest; 64 | }; 65 | 66 | 67 | deque* lock_table_[TABLE_SIZE]; 68 | 69 | // Queue of pointers to transactions that have acquired all locks that 70 | // they have requested. 'ready_txns_[key].front()' is the owner of the lock 71 | // for a specified key. 72 | // 73 | // Owned by the DeterministicScheduler. 74 | AtomicQueue* ready_txns_; 75 | 76 | // Configuration object (needed to avoid locking non-local keys). 77 | ClusterConfig* configuration_; 78 | 79 | // Tracks all txns still waiting on acquiring at least one lock. Entries in 80 | // 'txn_waits_' are invalided by any call to Release() with the entry's 81 | // txn. 82 | unordered_map txn_waits_; 83 | 84 | // mode_ == 0: origin CalvinDB; mode_ == 1: lowlatency CalvinDB 85 | uint32 mode_; 86 | }; 87 | #endif // _DB_SCHEDULER_DETERMINISTIC_LOCK_MANAGER_H_ 88 | -------------------------------------------------------------------------------- /src/scheduler/deterministic_scheduler.h: -------------------------------------------------------------------------------- 1 | // Author: Kun Ren 2 | // 3 | // 4 | // The deterministic lock manager implements deterministic locking as described 5 | // in 'The Case for Determinism in Database Systems', VLDB 2010. Each 6 | // transaction must request all locks it will ever need before the next 7 | // transaction in the specified order may acquire any locks. Each lock is then 8 | // granted to transactions in the order in which they requested them (i.e. in 9 | // the global transaction order). 10 | 11 | #ifndef _DB_SCHEDULER_DETERMINISTIC_SCHEDULER_H_ 12 | #define _DB_SCHEDULER_DETERMINISTIC_SCHEDULER_H_ 13 | 14 | #include 15 | #include 16 | #include 17 | #include 18 | #include 19 | #include 20 | #include 21 | #include 22 | #include 23 | 24 | #include "applications/application.h" 25 | #include "common/utils.h" 26 | #include "machine/connection.h" 27 | #include "backend/storage.h" 28 | #include "backend/storage_manager.h" 29 | #include "proto/message.pb.h" 30 | #include "proto/txn.pb.h" 31 | #include "scheduler/deterministic_lock_manager.h" 32 | #include "scheduler/scheduler.h" 33 | #include "proto/scalar.pb.h" 34 | 35 | using std::deque; 36 | using std::queue; 37 | 38 | class ClusterConfig; 39 | class Connection; 40 | class DeterministicLockManager; 41 | class Storage; 42 | class TxnProto; 43 | 44 | #define NUM_THREADS 3 45 | 46 | class DeterministicScheduler : public Scheduler { 47 | public: 48 | DeterministicScheduler(ClusterConfig* conf, Storage* storage, const Application* application,ConnectionMultiplexer* connection, uint32 mode); 49 | virtual ~DeterministicScheduler(); 50 | 51 | bool IsLocal(const Key& key); 52 | 53 | bool VerifyStorageCounters(TxnProto* txn, set>& keys); 54 | 55 | private: 56 | // Function for starting main loops in a separate pthreads. 57 | static void* WorkerThread(void* arg); 58 | 59 | static void* LockManagerThread(void* arg); 60 | 61 | // LockManager's main loop. 62 | void RunLockManagerThread(); 63 | 64 | // LockManager's main loop. 65 | void RunWorkerThread(uint32 thread); 66 | 67 | MessageProto* GetBatch(); 68 | 69 | // Configuration specifying node & system settings. 70 | ClusterConfig* configuration_; 71 | 72 | // Thread contexts and their associated Connection objects. 73 | pthread_t threads_[NUM_THREADS]; 74 | 75 | pthread_t lock_manager_thread_; 76 | 77 | // Storage layer used in application execution. 78 | Storage* storage_; 79 | 80 | // Application currently being run. 81 | const Application* application_; 82 | 83 | // The per-node lock manager tracks what transactions have temporary ownership 84 | // of what database objects, allowing the scheduler to track LOCAL conflicts 85 | // and enforce equivalence to transaction orders. 86 | DeterministicLockManager* lock_manager_; 87 | 88 | // Queue of transaction ids of transactions that have acquired all locks that 89 | // they have requested. 90 | AtomicQueue* ready_txns_; 91 | 92 | 93 | AtomicQueue* txns_queue_; 94 | AtomicQueue* done_queue_; 95 | 96 | // Connection for receiving txn batches from sequencer. 97 | ConnectionMultiplexer* connection_; 98 | 99 | bool start_working_; 100 | 101 | uint32 mode_; 102 | 103 | // below is for GetBatch 104 | unordered_map batches_data_; 105 | unordered_map global_batches_order_; 106 | 107 | Sequence* current_sequence_; 108 | uint32 current_sequence_id_; 109 | uint32 current_sequence_batch_index_; 110 | uint64 current_batch_id_; 111 | 112 | // Below is for request chopping with remaster transactions 113 | // pair 114 | map, vector> waiting_txns_by_key_; 115 | map>> waiting_txns_by_txnid_; 116 | map> blocking_txns_; 117 | 118 | }; 119 | #endif // _DB_SCHEDULER_DETERMINISTIC_SCHEDULER_H_ 120 | -------------------------------------------------------------------------------- /src/scheduler/lock_manager.h: -------------------------------------------------------------------------------- 1 | // Author: Kun Ren 2 | // 3 | // Interface for lock managers in the system. 4 | 5 | #ifndef _DB_SCHEDULER_LOCK_MANAGER_H_ 6 | #define _DB_SCHEDULER_LOCK_MANAGER_H_ 7 | 8 | #include 9 | 10 | #include "common/types.h" 11 | 12 | using std::vector; 13 | 14 | class TxnProto; 15 | 16 | // This interface supports locks being held in both read/shared and 17 | // write/exclusive modes. 18 | enum LockMode { 19 | UNLOCKED = 0, 20 | READ = 1, 21 | WRITE = 2, 22 | }; 23 | 24 | class LockManager { 25 | public: 26 | virtual ~LockManager() {} 27 | // Attempts to assign the lock for each key in keys to the specified 28 | // transaction. Returns the number of requested locks NOT assigned to the 29 | // transaction (therefore Lock() returns 0 if the transaction successfully 30 | // acquires all locks). 31 | // 32 | // Requires: 'read_keys' and 'write_keys' do not overlap, and neither contains 33 | // duplicate keys. 34 | // Requires: Lock has not previously been called with this txn_id. Note that 35 | // this means Lock can only ever be called once per txn. 36 | virtual int Lock(TxnProto* txn) = 0; 37 | 38 | // For each key in 'keys': 39 | // - If the specified transaction owns the lock on the item, the lock is 40 | // released. 41 | // - If the transaction is in the queue to acquire a lock on the item, the 42 | // request is cancelled and the transaction is removed from the item's 43 | // queue. 44 | virtual void Release(const Key& key, TxnProto* txn) = 0; 45 | virtual void Release(TxnProto* txn) = 0; 46 | 47 | // Locked sets '*owner' to contain the txn IDs of all txns holding the lock, 48 | // and returns the current state of the lock: UNLOCKED if it is not currently 49 | // held, READ or WRITE if it is, depending on the current state. 50 | virtual LockMode Status(const Key& key, vector* owners) = 0; 51 | }; 52 | #endif // _DB_SCHEDULER_LOCK_MANAGER_H_ 53 | -------------------------------------------------------------------------------- /src/scheduler/scheduler.h: -------------------------------------------------------------------------------- 1 | // Author: Kun Ren 2 | // 3 | // A database node's Scheduler determines what transactions should be run when 4 | // at that node. It is responsible for communicating with other nodes when 5 | // necessary to determine whether a transaction can be scheduled. It also 6 | // forwards messages on to the backend that are sent from other nodes 7 | // participating in distributed transactions. 8 | 9 | #ifndef _DB_SCHEDULER_SCHEDULER_H_ 10 | #define _DB_SCHEDULER_SCHEDULER_H_ 11 | 12 | class Application; 13 | 14 | class Scheduler { 15 | public: 16 | virtual ~Scheduler() {} 17 | 18 | }; 19 | #endif // _DB_SCHEDULER_SCHEDULER_H_ 20 | -------------------------------------------------------------------------------- /src/scripts/Makefile.inc: -------------------------------------------------------------------------------- 1 | # Create template specific variables 2 | UC_DIR := SCRIPTS 3 | LC_DIR := scripts 4 | 5 | SRCS := scripts/script_utils.cc 6 | 7 | EXES := scripts/cluster.cc \ 8 | scripts/calvindb_server.cc \ 9 | scripts/lowlatency_calvindb_server.cc 10 | 11 | DEPS := $(PROTO_OBJS) $(COMMON_OBJS) $(BACKEND_OBJS) $(APPLICATIONS_OBJS) $(MACHINE_OBJS) $(LOG_OBJS) $(SCHEDULER_OBJS) 12 | 13 | # Link the template to avoid redundancy 14 | include $(MAKEFILE_TEMPLATE) 15 | 16 | -------------------------------------------------------------------------------- /src/scripts/calvindb_server.cc: -------------------------------------------------------------------------------- 1 | // Author: Kun Ren 2 | // 3 | 4 | #include 5 | #include 6 | 7 | #include "machine/cluster_config.h" 8 | #include "log/local_mem_log.h" 9 | #include "log/paxos.h" 10 | #include "backend/simple_storage.h" 11 | #include "machine/connection.h" 12 | #include "machine/sequencer.h" 13 | #include "applications/microbenchmark.h" 14 | #include "applications/tpcc.h" 15 | #include "scheduler/deterministic_scheduler.h" 16 | #include "scripts/script_utils.h" 17 | 18 | DEFINE_bool(calvin_version, false, "Print Calvin version information"); 19 | DEFINE_string(binary, "calvindb_server", "Calvin binary executable program"); 20 | DEFINE_string(config, "calvin.conf", "conf file of Calvin cluster"); 21 | DEFINE_int32(machine_id, 0, "machine id"); 22 | DEFINE_int32(mode, 0, "0: Origin CalvinDB; 1: Low latency CalvinDB; 2; Low latency CalvinDB with access pattern remaster"); 23 | DEFINE_int32(type, 0, "[CalvinDB: 0: 3 replicas; 1: 6 replicas]; [Low latency: 0: 3 replicas normal; 1: 6 replicas normal; 2: 6 replicas strong availbility ] "); 24 | DEFINE_int32(experiment, 0, "the experiment that you want to run, default is microbenchmark"); 25 | DEFINE_int32(percent_mp, 0, "percent of distributed txns"); 26 | DEFINE_int32(percent_mr, 0, "percent of multi-replica txns"); 27 | DEFINE_int32(hot_records, 10000, "number of hot records--to control contention"); 28 | DEFINE_int32(max_batch_size, 100, "max batch size of txns per epoch"); 29 | 30 | int main(int argc, char** argv) { 31 | google::ParseCommandLineFlags(&argc, &argv, true); 32 | google::InitGoogleLogging(argv[0]); 33 | 34 | // Print Calvin version 35 | if (FLAGS_calvin_version) { 36 | // Check whether Calvin have been running 37 | if (is_process_exist((char *)FLAGS_binary.c_str()) == true) { 38 | return -2; 39 | } else { 40 | printf("Machine %d: (Geo-replicated CalvinDB) 0.1 (c) UMD 2017.\n", 41 | (int)FLAGS_machine_id); 42 | return 0; 43 | } 44 | } 45 | 46 | LOG(ERROR) <FromFile(FLAGS_config); 52 | 53 | LOG(ERROR)<(new MClient(config, FLAGS_percent_mp, FLAGS_hot_records)); 66 | } else { 67 | client = reinterpret_cast(new TClient(config, FLAGS_percent_mp, FLAGS_hot_records)); 68 | } 69 | 70 | Storage* storage; 71 | storage = new SimpleStorage(); 72 | 73 | LOG(ERROR) << FLAGS_machine_id<< ":Created storage "; 74 | 75 | Application* application = NULL; 76 | if (FLAGS_experiment == 0) { 77 | application = new Microbenchmark(config, multiplexer, FLAGS_hot_records); 78 | application->InitializeStorage(storage, config); 79 | } else { 80 | // Other benchmark 81 | application = new Tpcc(config, multiplexer, FLAGS_hot_records); 82 | application->InitializeStorage(storage, config); 83 | } 84 | 85 | LOG(ERROR) << FLAGS_machine_id << ":Created application "; 86 | 87 | // Create Paxos 88 | Paxos* paxos = NULL; 89 | if (FLAGS_machine_id % config->nodes_per_replica() == 0) { 90 | paxos = new Paxos(new LocalMemLog(), config, multiplexer); 91 | } 92 | 93 | LOG(ERROR) << FLAGS_machine_id << ":Created paxos log "; 94 | 95 | // Initialize sequencer component and start sequencer thread running. 96 | Sequencer sequencer(config, multiplexer, client, paxos, FLAGS_max_batch_size); 97 | 98 | LOG(ERROR) << FLAGS_machine_id << ":Created sequencer "; 99 | 100 | // Run scheduler in main thread. 101 | DeterministicScheduler scheduler(config, 102 | storage, 103 | application, 104 | multiplexer, 105 | FLAGS_mode); 106 | 107 | 108 | LOG(ERROR) << FLAGS_machine_id << ":Created scheduler "; 109 | 110 | while (!config->Stopped()) { 111 | usleep(1000000); 112 | } 113 | 114 | printf("Machine %d : Calvin server exit!\n", (int)FLAGS_machine_id); 115 | usleep(1000*1000); 116 | } 117 | 118 | -------------------------------------------------------------------------------- /src/scripts/cluster.cc: -------------------------------------------------------------------------------- 1 | // Author: Kun Ren 2 | // 3 | 4 | #include 5 | #include 6 | 7 | #include "machine/cluster_manager.h" 8 | 9 | DEFINE_string(command, "status", "cluster command"); 10 | DEFINE_string(config, "calvin.conf", "conf file of Calvin cluster"); 11 | DEFINE_string(calvin_path, "/home/ubuntu/CalvinDB", "path to the main calvin directory"); 12 | DEFINE_string(binary, "calvindb_server", "Calvin binary executable program"); 13 | DEFINE_string(lowlatency_binary, "lowlatency_calvindb_server", "Lowlatency Calvin binary executable program"); 14 | DEFINE_string(ssh_key1, "-i ~/Virginia.pem", "ssh_key for the first data center(Virginia)"); 15 | DEFINE_string(ssh_key2, "-i ~/Oregon.pem", "ssh_key for the second data center(Oregon)"); 16 | DEFINE_string(ssh_key3, "-i ~/Ireland.pem", "ssh_key for the third data center(Ireland)"); 17 | DEFINE_string(ssh_key4, "-i ~/Ohio.pem", "ssh_key for the first data center(Virginia)"); 18 | DEFINE_string(ssh_key5, "-i ~/California.pem", "ssh_key for the second data center(Oregon)"); 19 | DEFINE_string(ssh_key6, "-i ~/London.pem", "ssh_key for the third data center(Ireland)"); 20 | DEFINE_int32(lowlatency, 0, "0: Original CalvinDB ; 1: low latency version of CalvinDB; 2: low latency with access pattern remasters"); 21 | DEFINE_int32(type, 0, "[CalvinDB: 0: 3 replicas; 1: 6 replicas]; [Low latency: 0: 3 replicas normal; 1: 6 replicas normal; 2: 6 replicas strong availbility ] "); 22 | DEFINE_int32(experiment, 0, "the experiment that you want to run, default is microbenchmark"); 23 | DEFINE_int32(percent_mp, 0, "percent of distributed txns"); 24 | DEFINE_int32(percent_mr, 0, "percent of multi-replica txns"); 25 | DEFINE_int32(hot_records, 10000, "number of hot records--to control contention"); 26 | DEFINE_int32(max_batch_size, 100, "max batch size of txns per epoch"); 27 | 28 | int main(int argc, char** argv) { 29 | google::ParseCommandLineFlags(&argc, &argv, true); 30 | 31 | ClusterManager* cm; 32 | if (FLAGS_lowlatency == 0) { 33 | if (FLAGS_type == 0) { 34 | // 3 replicas original CalvinDB 35 | cm = new ClusterManager(FLAGS_config, FLAGS_calvin_path, FLAGS_binary, FLAGS_lowlatency, FLAGS_type, FLAGS_ssh_key1, FLAGS_ssh_key2, FLAGS_ssh_key3); 36 | } else { 37 | cm = new ClusterManager(FLAGS_config, FLAGS_calvin_path, FLAGS_binary, FLAGS_lowlatency, FLAGS_type, FLAGS_ssh_key1, FLAGS_ssh_key2, FLAGS_ssh_key3, 38 | FLAGS_ssh_key4, FLAGS_ssh_key5, FLAGS_ssh_key6); 39 | } 40 | } else { 41 | if (FLAGS_type == 0) { 42 | cm = new ClusterManager(FLAGS_config, FLAGS_calvin_path, FLAGS_lowlatency_binary, FLAGS_lowlatency, FLAGS_type, FLAGS_ssh_key1, FLAGS_ssh_key2, FLAGS_ssh_key3); 43 | } else { 44 | cm = new ClusterManager(FLAGS_config, FLAGS_calvin_path, FLAGS_lowlatency_binary, FLAGS_lowlatency, FLAGS_type, FLAGS_ssh_key1, FLAGS_ssh_key2, FLAGS_ssh_key3, 45 | FLAGS_ssh_key4, FLAGS_ssh_key5, FLAGS_ssh_key6); 46 | } 47 | } 48 | 49 | if (FLAGS_command == "update") { 50 | cm->Update(); 51 | 52 | } else if (FLAGS_command == "put-config") { 53 | cm->PutConfig(); 54 | 55 | } else if (FLAGS_command == "get-data") { 56 | cm->GetTempFiles("report."); 57 | 58 | } else if (FLAGS_command == "start") { 59 | cm->DeployCluster(FLAGS_experiment, FLAGS_percent_mp, FLAGS_percent_mr, FLAGS_hot_records, FLAGS_max_batch_size); 60 | 61 | } else if (FLAGS_command == "kill") { 62 | cm->KillCluster(); 63 | 64 | } else if (FLAGS_command == "status") { 65 | cm->ClusterStatus(); 66 | 67 | } else { 68 | LOG(FATAL) << "unknown command: " << FLAGS_command; 69 | } 70 | return 0; 71 | } 72 | 73 | -------------------------------------------------------------------------------- /src/scripts/lowlatency_calvindb_server.cc: -------------------------------------------------------------------------------- 1 | // Author: Kun Ren 2 | // 3 | 4 | #include 5 | #include 6 | 7 | #include "machine/cluster_config.h" 8 | #include "log/paxos.h" 9 | #include "backend/simple_storage.h" 10 | #include "machine/connection.h" 11 | #include "machine/lowlatency_sequencer.h" 12 | #include "applications/microbenchmark.h" 13 | #include "applications/tpcc.h" 14 | #include "scheduler/deterministic_scheduler.h" 15 | #include "scripts/script_utils.h" 16 | 17 | DEFINE_bool(calvin_version, false, "Print Calvin version information"); 18 | DEFINE_string(binary, "lowlatency_calvindb_server", "Lowlatency Calvin binary executable program"); 19 | DEFINE_string(config, "calvin.conf", "conf file of Calvin cluster"); 20 | DEFINE_int32(machine_id, 0, "machine id"); 21 | DEFINE_int32(mode, 1, "0: Origin CalvinDB; 1: Low latency CalvinDB; 2; Low latency CalvinDB with access pattern remaster"); 22 | DEFINE_int32(type, 0, "[CalvinDB: 0: 3 replicas; 1: 6 replicas]; [Low latency: 0: 3 replicas normal; 1: 6 replicas normal; 2: 6 replicas strong availbility ] "); 23 | DEFINE_int32(experiment, 0, "the experiment that you want to run, default is microbenchmark"); 24 | DEFINE_int32(percent_mp, 0, "percent of distributed txns"); 25 | DEFINE_int32(percent_mr, 0, "percent of multi-replica txns"); 26 | DEFINE_int32(hot_records, 10000, "number of hot records--to control contention"); 27 | DEFINE_int32(max_batch_size, 100, "max batch size of txns per epoch"); 28 | 29 | int main(int argc, char** argv) { 30 | google::ParseCommandLineFlags(&argc, &argv, true); 31 | google::InitGoogleLogging(argv[0]); 32 | 33 | // Print Calvin version 34 | if (FLAGS_calvin_version) { 35 | // Check whether Calvin have been running 36 | if (is_process_exist((char *)FLAGS_binary.c_str()) == true) { 37 | return -2; 38 | } else { 39 | printf("Machine %d: (Geo-replicated CalvinDB) 0.1 (c) UMD 2017.\n", 40 | (int)FLAGS_machine_id); 41 | return 0; 42 | } 43 | } 44 | 45 | LOG(ERROR) <FromFile(FLAGS_config); 51 | 52 | LOG(ERROR)<(new Lowlatency_MClient(config, FLAGS_percent_mp, FLAGS_percent_mr, FLAGS_hot_records)); 65 | } else { 66 | client = reinterpret_cast(new Lowlatency_TClient(config, FLAGS_percent_mp, FLAGS_percent_mr, FLAGS_hot_records)); 67 | } 68 | 69 | Storage* storage; 70 | storage = new SimpleStorage(); 71 | 72 | LOG(ERROR) << FLAGS_machine_id<< ":Created storage "; 73 | 74 | Application* application = NULL; 75 | if (FLAGS_experiment == 0) { 76 | application = new Microbenchmark(config, multiplexer, FLAGS_hot_records); 77 | application->InitializeStorage(storage, config); 78 | } else { 79 | // Other benchmark 80 | application = new Tpcc(config, multiplexer, FLAGS_hot_records); 81 | application->InitializeStorage(storage, config); 82 | } 83 | 84 | LOG(ERROR) << FLAGS_machine_id << ":Created application "; 85 | 86 | // Create Paxos 87 | LocalPaxos* paxos = NULL; 88 | if (FLAGS_machine_id % config->nodes_per_replica() < 3) { 89 | paxos = new LocalPaxos(config, multiplexer, FLAGS_type); 90 | } 91 | 92 | LOG(ERROR) << FLAGS_machine_id << ":Created paxos log "; 93 | 94 | // Initialize sequencer component and start sequencer thread running. 95 | LowlatencySequencer sequencer(config, multiplexer, client, paxos, storage, FLAGS_max_batch_size); 96 | 97 | LOG(ERROR) << FLAGS_machine_id << ":Created sequencer "; 98 | 99 | // Run scheduler in main thread. 100 | DeterministicScheduler scheduler(config, 101 | storage, 102 | application, 103 | multiplexer, 104 | FLAGS_mode); 105 | 106 | LOG(ERROR) << FLAGS_machine_id << ":Created scheduler "; 107 | 108 | while (!config->Stopped()) { 109 | usleep(1000000); 110 | } 111 | 112 | printf("Machine %d : Calvin server exit!\n", (int)FLAGS_machine_id); 113 | usleep(1000*1000); 114 | } 115 | 116 | -------------------------------------------------------------------------------- /src/scripts/script_utils.cc: -------------------------------------------------------------------------------- 1 | // Author: Kun Ren 2 | // 3 | 4 | #include "scripts/script_utils.h" 5 | 6 | #include 7 | #include 8 | #include 9 | #include 10 | #include 11 | #include 12 | #include 13 | #include 14 | #include 15 | #include 16 | 17 | string HostName() { 18 | char hname[128]; 19 | gethostname(hname, sizeof(hname)); 20 | struct hostent* hent = gethostbyname(hname); 21 | return hent->h_name; 22 | } 23 | 24 | // Get the basename of the process name 25 | char *basename(char *path) { 26 | char *s; 27 | char *p; 28 | p = s = path; 29 | while (*s) { 30 | if (*s++ == '/') { 31 | p = s; 32 | } 33 | } 34 | return (char *) p; 35 | } 36 | 37 | // find all pid of process by name, only compare base name of pid_name 38 | // list_size: the size of pid_list 39 | // RETURN: 40 | // < 0: error number 41 | // >=0: how many pid found 42 | int get_pid_by_name(char* process_name, int list_size) { 43 | #define MAX_BUF_SIZE 256 44 | DIR *dir; 45 | struct dirent *next; 46 | int count = 0; 47 | pid_t pid; 48 | FILE *fp; 49 | char *base_pname = NULL; 50 | char *base_fname = NULL; 51 | char cmdline[MAX_BUF_SIZE]; 52 | char path[MAX_BUF_SIZE]; 53 | 54 | if (process_name == NULL) { 55 | return -EINVAL; 56 | } 57 | 58 | base_pname = basename(process_name); 59 | if (strlen(base_pname) <= 0) { 60 | return -EINVAL; 61 | } 62 | 63 | dir = opendir("/proc"); 64 | if (!dir) { 65 | return -EIO; 66 | } 67 | 68 | while ((next = readdir(dir)) != NULL) { 69 | // Skip non-number 70 | if (!isdigit(*next->d_name)) { 71 | continue; 72 | } 73 | 74 | pid = strtol(next->d_name, NULL, 0); 75 | snprintf(path, sizeof(path), "/proc/%u/cmdline", pid); 76 | fp = fopen(path, "r"); 77 | if (fp == NULL) { 78 | continue; 79 | } 80 | 81 | memset(cmdline, 0, sizeof(cmdline)); 82 | size_t result = fread(cmdline, MAX_BUF_SIZE - 1, 1, fp); 83 | if (result < 0) { 84 | fclose(fp); 85 | continue; 86 | } 87 | fclose(fp); 88 | base_fname = basename(cmdline); 89 | 90 | if (strcmp(base_fname, base_pname) == 0) { 91 | if (count >= list_size) { 92 | break; 93 | } else { 94 | count++; 95 | } 96 | } 97 | } 98 | closedir(dir); 99 | return count; 100 | } 101 | 102 | // If process is existed, return true 103 | bool is_process_exist(char* process_name) { 104 | int count = get_pid_by_name(process_name, 2); 105 | if (count > 1) { 106 | return true; 107 | } else { 108 | return false; 109 | } 110 | } 111 | 112 | -------------------------------------------------------------------------------- /src/scripts/script_utils.h: -------------------------------------------------------------------------------- 1 | // Author: Kun Ren 2 | // 3 | 4 | #ifndef CALVIN_SCRIPTS_SCRIPT_UTILS_H_ 5 | #define CALVIN_SCRIPTS_SCRIPT_UTILS_H_ 6 | 7 | #include 8 | using std::string; 9 | 10 | bool is_process_exist(char* process_name); 11 | 12 | string HostName(); 13 | 14 | #endif // CALVIN_SCRIPTS_SCRIPT_UTILS_H_ 15 | 16 | --------------------------------------------------------------------------------