├── .github └── workflows │ └── go.yml ├── .gitignore ├── Dockerfile.test ├── LICENSE ├── Makefile ├── README.md ├── changelog ├── changelog.go ├── mysql.go └── mysql_test.go ├── codecov.yaml ├── config ├── base.yaml ├── config.go ├── config_test.go ├── development.yaml ├── mysql.go ├── production.yaml ├── std.go ├── test.yaml ├── topic.go └── topic_test.go ├── db ├── builtin.go ├── gtid.go ├── gtid_test.go ├── open.go ├── open_test.go ├── resolver.go └── test.go ├── debian ├── changelog ├── compat ├── control ├── copyright ├── rules ├── source │ └── format ├── storagetapper-client.install ├── storagetapper-server.install ├── storagetapper-server.postinst └── storagetapper-server.service ├── doc ├── commonformat.md ├── endpoints.md └── options.md ├── docker_test_entry.sh ├── encoder ├── avro.go ├── avro_schema.go ├── avro_schema_test.go ├── encoder.go ├── encoder_test.go ├── json.go ├── msgpack.go ├── sql.go ├── sql_test.go └── z.go ├── go.mod ├── go.sum ├── lock ├── lock.go └── lock_test.go ├── log ├── log.go ├── log_test.go ├── rus.go ├── rus_test.go ├── std.go ├── std_test.go ├── zap.go └── zap_test.go ├── main.go ├── main_test.go ├── metrics ├── counter.go ├── metrics.go ├── metrics_test.go ├── noop.go ├── process.go ├── tally.go └── timer.go ├── pipe ├── cache.go ├── cache_test.go ├── file.go ├── file_test.go ├── hdfs.go ├── hdfs_test.go ├── header.go ├── kafka.go ├── kafka_test.go ├── local.go ├── local_test.go ├── pipe.go ├── pipe_test.go ├── s3.go ├── s3_test.go ├── sql.go └── sql_test.go ├── pool ├── pool.go └── pool_test.go ├── pprof_test.go ├── schema ├── alter_schema.go ├── alter_test.go ├── mysql_avro_schema_converter.go ├── schema_test.go └── table_schema.go ├── scripts ├── docker_test_entry.sh ├── install_deps.sh ├── install_go.sh ├── install_hadoop.sh ├── install_hive.sh ├── install_kafka.sh ├── install_mysql.sh ├── install_sql.sh ├── prepare_test_env.sh ├── run_benchmarks.sh ├── run_lints.sh ├── run_tests.sh ├── start_kafka.sh ├── stcli ├── stop_kafka.sh └── workflow_example.sh ├── server ├── cluster_handler.go ├── cluster_test.go ├── config.go ├── config_test.go ├── index.go ├── schema_handler.go ├── schema_test.go ├── server.go ├── server_test.go ├── table_handler.go └── table_test.go ├── shutdown ├── shutdown.go └── shutdown_test.go ├── snapshot ├── mysql.go ├── mysql_test.go ├── snapshot.go └── snapshot_test.go ├── state ├── clusters.go ├── clusters_test.go ├── lock.go ├── lock_test.go ├── manager.go ├── schema.go ├── state.go └── state_test.go ├── streamer ├── buffer.go ├── snapshot.go ├── streamer.go └── streamer_test.go ├── test ├── env.go ├── env_test.go ├── wait.go └── wait_test.go ├── throttle ├── throttle.go └── throttle_test.go ├── types ├── avro.go ├── const.go ├── format.go ├── input.go ├── schema.go └── types.go └── util ├── mysql.go ├── mysql_test.go ├── util.go └── util_test.go /.github/workflows/go.yml: -------------------------------------------------------------------------------- 1 | name: Go 2 | on: [push] 3 | jobs: 4 | 5 | build: 6 | name: Test 7 | runs-on: ubuntu-latest 8 | steps: 9 | 10 | - name: Check out code 11 | uses: actions/checkout@v1 12 | 13 | - name: Fix permissions 14 | run: chmod +x ./scripts/install_deps.sh 15 | 16 | - name: Install dependencies 17 | run: DOCKER_BUILD=1 /bin/sh scripts/install_deps.sh 18 | 19 | - name: Prepare test environment 20 | run: sudo /bin/sh scripts/prepare_test_env.sh 21 | 22 | - name: Start S3 docker container 23 | run: docker run --cidfile=/tmp/s3server.cid -d -p 8000:8000 scality/s3server 24 | 25 | - name: Run tests 26 | run: GOROOT="/usr/local/go" GOPATH="$HOME/gopath" PATH="/usr/local/go/bin:$GOPATH/bin:$PATH" make test 27 | -------------------------------------------------------------------------------- /.gitignore: -------------------------------------------------------------------------------- 1 | *.out 2 | *.test 3 | *.xml 4 | *.swp 5 | .idea/ 6 | *.iml 7 | *.cov 8 | *.html 9 | .tmp/ 10 | .gen/ 11 | /vendor 12 | /storagetapper 13 | types/format_gen*.go 14 | .DS_Store 15 | -------------------------------------------------------------------------------- /Dockerfile.test: -------------------------------------------------------------------------------- 1 | FROM debian:stretch 2 | 3 | RUN apt-get update && apt-get install -y apt-transport-https 4 | 5 | RUN echo "deb http://deb.debian.org/debian stretch main contrib\n \ 6 | deb http://deb.debian.org/debian-security stretch/updates main" > /etc/apt/sources.list && apt-get update && apt-get -y --force-yes dist-upgrade 7 | 8 | RUN DEBIAN_FRONTEND="noninteractive" apt-get install -y --force-yes lsb-release wget default-jre sudo vim openssh-server curl netcat build-essential 9 | 10 | COPY scripts/install_mysql.sh /scripts/ 11 | RUN GOPATH=$HOME/gopath PATH=$HOME/gopath/bin:$PATH DOCKER_BUILD=1 /bin/sh scripts/install_mysql.sh 12 | COPY scripts/install_go.sh /scripts/ 13 | RUN GOPATH=$HOME/gopath PATH=$HOME/gopath/bin:$PATH DOCKER_BUILD=1 /bin/sh scripts/install_go.sh 14 | COPY scripts/start_kafka.sh /scripts/ 15 | COPY scripts/install_kafka.sh /scripts/ 16 | RUN GOPATH=$HOME/gopath PATH=$HOME/gopath/bin:$PATH DOCKER_BUILD=1 /bin/sh scripts/install_kafka.sh 17 | COPY scripts/install_hadoop.sh /scripts/ 18 | RUN GOPATH=$HOME/gopath PATH=$HOME/gopath/bin:$PATH DOCKER_BUILD=1 /bin/sh scripts/install_hadoop.sh 19 | COPY scripts/install_sql.sh /scripts/ 20 | RUN GOPATH=$HOME/gopath PATH=$HOME/gopath/bin:$PATH DOCKER_BUILD=1 /bin/sh scripts/install_sql.sh 21 | COPY scripts/install_hive.sh /scripts/ 22 | RUN GOPATH=$HOME/gopath PATH=$HOME/gopath/bin:$PATH DOCKER_BUILD=1 /bin/sh scripts/install_hive.sh 23 | 24 | COPY scripts/prepare_test_env.sh /scripts/ 25 | 26 | COPY scripts/docker_test_entry.sh / 27 | RUN chmod +x /docker_test_entry.sh 28 | 29 | RUN echo 'set -x;sh scripts/prepare_test_env.sh\nexport GOPATH=~/gopath;export PATH="$PATH:/usr/local/go/bin:$GOPATH/bin"\nexport STORAGETAPPER_ENVIRONMENT=test' >/root/.bashrc 30 | 31 | WORKDIR /storagetapper 32 | 33 | ENV USER=root 34 | RUN echo "export PATH=$PATH:/root/gopath/bin:/usr/local/go/bin" > /root/.bashrc 35 | 36 | CMD /bin/bash -l 37 | -------------------------------------------------------------------------------- /LICENSE: -------------------------------------------------------------------------------- 1 | Copyright (c) 2017 Uber Technologies, Inc. 2 | 3 | Permission is hereby granted, free of charge, to any person obtaining a copy 4 | of this software and associated documentation files (the "Software"), to deal 5 | in the Software without restriction, including without limitation the rights 6 | to use, copy, modify, merge, publish, distribute, sublicense, and/or sell 7 | copies of the Software, and to permit persons to whom the Software is 8 | furnished to do so, subject to the following conditions: 9 | 10 | The above copyright notice and this permission notice shall be included in 11 | all copies or substantial portions of the Software. 12 | 13 | THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR 14 | IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, 15 | FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE 16 | AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER 17 | LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, 18 | OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN 19 | THE SOFTWARE. 20 | -------------------------------------------------------------------------------- /Makefile: -------------------------------------------------------------------------------- 1 | NAME := storagetapper 2 | GIT_REVISION := $(shell git rev-parse --short HEAD) 3 | 4 | PKGS := $(shell go list ./...|sed 's+github.com/uber/storagetapper+.+g'|sort -r) 5 | SRCS := $(shell find . -name "*.go" -not -path './vendor') 6 | 7 | $(NAME): $(SRCS) types/format_gen.go 8 | go build -ldflags "-X main.revision=$(GIT_REVISION)" 9 | 10 | install: $(NAME) 11 | install -m 0755 -d $(DESTDIR)/usr/bin 12 | install -m 0550 -s $(NAME) $(DESTDIR)/usr/bin 13 | install -m 0750 -d $(DESTDIR)/etc/$(NAME) 14 | install -m 0600 config/base.yaml config/production.yaml $(DESTDIR)/etc/$(NAME) 15 | 16 | types/format_gen.go: 17 | go generate ./types 18 | 19 | unittest: $(NAME) 20 | sh scripts/run_tests.sh $(PKGS) 21 | 22 | lint: $(NAME) 23 | sh scripts/run_lints.sh $(PKGS) 24 | 25 | shellcheck: $(NAME) 26 | shellcheck scripts/*.sh 27 | 28 | bench: $(NAME) 29 | sh scripts/run_benchmarks.sh $(PKGS) 30 | 31 | test: unittest lint shellcheck 32 | 33 | deb: 34 | dpkg-buildpackage -uc -us -b 35 | 36 | clean: 37 | rm -f $(NAME) 38 | 39 | docker-image: 40 | docker build -f Dockerfile.test -t uber/storagetapper_test . 41 | 42 | docker-test: docker-image 43 | -docker rm -f $(shell cat /tmp/s3server.cid) 44 | rm -f /tmp/s3server.cid 45 | docker run --cidfile=/tmp/s3server.cid -d -p 8000:8000 scality/s3server 46 | docker run --cap-add sys_nice --cap-add net_admin --cap-add ipc_lock -v $(shell pwd):/storagetapper uber/storagetapper_test /bin/bash /docker_test_entry.sh 47 | 48 | test-env: docker-image 49 | docker run --cap-add sys_nice --cap-add net_admin --cap-add ipc_lock -it -v $(shell pwd):/storagetapper -e "GOPATH=/root/gopath" -w /storagetapper/ uber/storagetapper_test 50 | -------------------------------------------------------------------------------- /README.md: -------------------------------------------------------------------------------- 1 | StorageTapper 2 | ------------- 3 | 4 | Overview 5 | -------- 6 | 7 | [![Build Status](https://github.com/uber/storagetapper/workflows/Go/badge.svg)]() 8 | [![Go Report Card](https://goreportcard.com/badge/github.com/uber/storagetapper)](https://goreportcard.com/report/github.com/uber/storagetapper) 9 | [![codecov](https://codecov.io/gh/uber/storagetapper/branch/master/graph/badge.svg)](https://codecov.io/gh/uber/storagetapper) 10 | 11 | StorageTapper is a scalable realtime MySQL change data streaming, logical backup 12 | and logical replication service. 13 | 14 | Storagetapper is deployed in production at Uber and used to produce snapshot and 15 | realtime changed data of thousands of MySQL tables across multiple datacenters. 16 | 17 | It is also used as a backup service to snapshot hundreds of terrabytes 18 | of Schemaless data to HDFS and S3 with optional asymmetric encryption and 19 | compression. 20 | 21 | It reads data from source transforms according to the specified event 22 | format and produces data to destination. 23 | 24 | Supported event sources: 25 | * MySQL 26 | * Schemaless 27 | 28 | Supported event destinations: 29 | * Kafka 30 | * HDFS 31 | * S3 32 | * Local file 33 | * MySQL (experimental) 34 | * Postgres (experimental) 35 | * Clickhouse (experimental) 36 | 37 | Supported event formats: 38 | * Avro 39 | * [JSON](./doc/commonformat.md) 40 | * MsgPack 41 | * SQL 42 | 43 | Features: 44 | * Fields filtering 45 | * Snapshot row filtering 46 | * Producing realtime changes only, i.e. omitting initial snapshot 47 | * Snapshot and changelog events ordering 48 | * Periodic snapshots 49 | * Snapshot index hinting 50 | 51 | Storagetapper keeps its jobs state in MySQL database and automatically distributes jobs 52 | between configured number of workers. 53 | 54 | It is also aware of node roles and takes snapshot from the slave nodes in order 55 | to reduce load on master nodes. StorageTapper can also optionally further throttle its reads. 56 | Binlogs are streamed from master nodes for better SLAs. 57 | 58 | Service is dynamically configurable through [RESTful API](./doc/endpoints.md) or 59 | built-in UI. 60 | 61 | Build & Install 62 | --------------- 63 | 64 | ## Debian & Ubuntu 65 | ```sh 66 | cd storagetapper 67 | make deb && dpkg -i ../storagetapper_1.0_amd64.deb 68 | ``` 69 | 70 | ## Others 71 | ```sh 72 | cd storagetapper 73 | make && make install 74 | ``` 75 | 76 | ## Development 77 | 78 | ### Linux 79 | 80 | ```sh 81 | /bin/bash scripts/install_deps.sh # install all dependencies: MySQL, Kafka, HDFS, S3, ... 82 | make test # run all tests 83 | GO111MODULE=on TEST_PARAM="-test.run=TestLocalBasic" /bin/bash scripts/run_tests.sh ./pipe # individual test 84 | ``` 85 | 86 | ### Non Linux 87 | ```sh 88 | make test-env 89 | $ make test 90 | ``` 91 | 92 | Configuration 93 | ------------- 94 | 95 | Storagetapper loads configuration from the following files and location in the 96 | given order: 97 | ```sh 98 | /etc/storagetapper/base.yaml 99 | /etc/storagetapper/production.yaml 100 | $(HOME)/base.yaml 101 | $(HOME)/production.yaml 102 | $(STORAGETAPPER_CONFIG_DIR)/base.yaml 103 | $(STORAGETAPPER_CONFIG_DIR)/production.yaml 104 | ``` 105 | 106 | Available options described in [Configuration](./doc/options.md) section 107 | 108 | License 109 | ------- 110 | This software is licensed under the [MIT License](LICENSE). 111 | 112 | -------------------------------------------------------------------------------- /changelog/changelog.go: -------------------------------------------------------------------------------- 1 | package changelog 2 | 3 | import ( 4 | "github.com/uber/storagetapper/config" 5 | "github.com/uber/storagetapper/log" 6 | "github.com/uber/storagetapper/pipe" 7 | "github.com/uber/storagetapper/pool" 8 | "golang.org/x/net/context" //"context" 9 | ) 10 | 11 | //Reader is a contract for changelog reader 12 | type Reader interface { 13 | //Worker is a main log reader routine 14 | //returns false if no more log readers of this type are needed 15 | Worker() bool 16 | } 17 | 18 | //ReaderConstructor initializes logger plugin 19 | type ReaderConstructor func(ctx context.Context, cfg *config.AppConfig, bufPipe pipe.Pipe, t pool.Thread) (Reader, error) 20 | 21 | //Plugins contains registered binlog reader plugins 22 | var Plugins map[string]ReaderConstructor 23 | 24 | func registerPlugin(name string, init ReaderConstructor) { 25 | if Plugins == nil { 26 | Plugins = make(map[string]ReaderConstructor) 27 | } 28 | Plugins[name] = init 29 | } 30 | 31 | //Worker iterates over available workers and try start them 32 | func Worker(ctx context.Context, cfg *config.AppConfig, bufPipe pipe.Pipe, tp pool.Thread) bool { 33 | for n, init := range Plugins { 34 | reader, err := init(ctx, cfg, bufPipe, tp) 35 | if err != nil { 36 | log.Errorf("Init failed for: %v", n) 37 | } 38 | if reader.Worker() { 39 | return true 40 | } 41 | } 42 | return false 43 | } 44 | -------------------------------------------------------------------------------- /codecov.yaml: -------------------------------------------------------------------------------- 1 | ignore: 2 | - test/*.go 3 | - util/*.go 4 | - types/*.go 5 | -------------------------------------------------------------------------------- /config/base.yaml: -------------------------------------------------------------------------------- https://raw.githubusercontent.com/uber/storagetapper/eed2574477ed96d7d063be5defa1319e8a7cbf60/config/base.yaml -------------------------------------------------------------------------------- /config/development.yaml: -------------------------------------------------------------------------------- 1 | log_type: zap 2 | 3 | logging: 4 | level: debug 5 | 6 | state_update_interval: 10s 7 | worker_idle_interval: 1s 8 | state_connect_url: "storagetapper:storagetapper@localhost" 9 | 10 | pipe: 11 | kafka: 12 | addresses: 13 | - "localhost:9091" 14 | - "localhost:9092" 15 | - "localhost:9093" 16 | hadoop: 17 | user: hadoop 18 | addresses: 19 | - "localhost:9000" 20 | base_dir: "/user/hadoop/tmp/hdfs_pipe_test" 21 | s3: 22 | region: "us-east-1" 23 | endpoint: "http://localhost:8000" 24 | access_key_id: 'accessKey1' 25 | secret_access_key: 'verySecretKey1' 26 | bucket: "pipetestbucket" 27 | base_dir: "/tmp/s3_pipe_test" 28 | 29 | output_topic_name_template: 30 | mysql: 31 | mysql: 32 | "{{.DB}}.{{.Table}}" 33 | -------------------------------------------------------------------------------- /config/production.yaml: -------------------------------------------------------------------------------- 1 | log_type: zap 2 | log_level: debug 3 | 4 | changelog_topic_name_template_default: "storagetapper-{{.Service}}-{{.DB}}-{{.Table}}{{if .Version}}-v{{.Version}}{{end}}" 5 | 6 | output_topic_name_template: 7 | mysql: 8 | kafka: 9 | "hp-dbevents-mysql-{{.Service}}-{{.DB}}-{{.Table}}" 10 | hdfs: "{{.Input}}/{{.Service}}/{{.Table}}/{{.Version}}/{{.DB}}_" 11 | s3: "{{.Input}}/{{.Service}}/{{.Table}}/{{.Version}}/{{.DB}}_" 12 | 13 | max_file_size: 5368709120 14 | -------------------------------------------------------------------------------- /config/std.go: -------------------------------------------------------------------------------- 1 | // Copyright (c) 2017 Uber Technologies, Inc. 2 | // 3 | // Permission is hereby granted, free of charge, to any person obtaining a copy 4 | // of this software and associated documentation files (the "Software"), to deal 5 | // in the Software without restriction, including without limitation the rights 6 | // to use, copy, modify, merge, publish, distribute, sublicense, and/or sell 7 | // copies of the Software, and to permit persons to whom the Software is 8 | // furnished to do so, subject to the following conditions: 9 | // 10 | // The above copyright notice and this permission notice shall be included in 11 | // all copies or substantial portions of the Software. 12 | // 13 | // THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR 14 | // IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, 15 | // FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE 16 | // AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER 17 | // LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, 18 | // OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN 19 | // THE SOFTWARE. 20 | 21 | package config 22 | 23 | import ( 24 | "fmt" 25 | "io/ioutil" 26 | "os" 27 | "strings" 28 | 29 | "gopkg.in/yaml.v2" 30 | 31 | "github.com/uber/storagetapper/types" 32 | ) 33 | 34 | const defaultEnvironment = EnvProduction 35 | const defaultZone = "" 36 | 37 | var paths = []string{"/etc/" + types.MySvcName, os.Getenv("HOME") + "/." + types.MySvcName, os.Getenv(strings.ToUpper(types.MySvcName) + "_CONFIG_DIR")} 38 | 39 | var exit = os.Exit 40 | 41 | type std struct { 42 | loadFn func(interface{}, string) ([]byte, error) 43 | saveFn func(interface{}, string, []byte) error 44 | env string 45 | } 46 | 47 | func (c *std) getEnvironment() string { 48 | if len(c.env) != 0 { 49 | return c.env 50 | } 51 | c.env = strings.ToLower(os.Getenv(strings.ToUpper(types.MySvcName) + "_ENVIRONMENT")) 52 | if c.env != EnvProduction && c.env != EnvDevelopment && c.env != EnvStaging && c.env != EnvTest { 53 | c.env = defaultEnvironment 54 | } 55 | return c.env 56 | } 57 | 58 | //environment returns current environment 59 | func (c *std) environment() string { 60 | return c.getEnvironment() 61 | } 62 | 63 | // Zone returns the current zone that the application is running in 64 | func (c *std) zone() string { 65 | return defaultZone 66 | } 67 | 68 | func stdReadFile(_ interface{}, name string) ([]byte, error) { 69 | return ioutil.ReadFile(name) 70 | } 71 | 72 | func (c *std) loadFile(file string, cfg interface{}) error { 73 | b, err := c.loadFn(cfg, file) 74 | if err != nil { 75 | if os.IsNotExist(err) { 76 | return nil 77 | } 78 | return fmt.Errorf("error loading: %v: %v", file, err.Error()) 79 | } 80 | log("config file: %+v\n", file) 81 | if err = yaml.Unmarshal(b, cfg); err != nil { 82 | return fmt.Errorf("error parsing: %v: %v", file, err.Error()) 83 | } 84 | return nil 85 | } 86 | 87 | func (c *std) saveFile(file string, cfg interface{}) error { 88 | var b []byte 89 | var err error 90 | if c.saveFn == nil { 91 | return nil 92 | } 93 | if b, err = yaml.Marshal(cfg); err != nil { 94 | return fmt.Errorf("error serializing: %v: %v", file, err.Error()) 95 | } 96 | err = c.saveFn(cfg, file, b) 97 | if err != nil { 98 | return fmt.Errorf("error saving: %v: %v", file, err.Error()) 99 | } 100 | log("saved config file: %+v\n", file) 101 | return nil 102 | } 103 | 104 | func (c *std) loadPath(path string, cfg interface{}) error { 105 | if err := c.loadFile(path+"/base.yaml", cfg); err != nil { 106 | return err 107 | } 108 | return c.loadFile(path+"/"+c.env+".yaml", cfg) 109 | } 110 | 111 | func (c *std) loadSection(cfg interface{}) error { 112 | c.getEnvironment() 113 | for _, v := range paths { 114 | if v == "" { 115 | continue 116 | } 117 | if err := c.loadPath(v, cfg); err != nil { 118 | return err 119 | } 120 | } 121 | log("config: %+v\n", cfg) 122 | return nil 123 | } 124 | 125 | func (c *std) parseConfig(cfg *AppConfig) error { 126 | if cfg.PortDyn == 0 { 127 | cfg.PortDyn = cfg.Port 128 | } 129 | return nil 130 | } 131 | 132 | func (c *std) load(cfg *AppConfig) error { 133 | return c.loadSection(&cfg.AppConfigODS) 134 | } 135 | 136 | func (c *std) save(cfg *AppConfig) error { 137 | return c.saveFile(paths[len(paths)-1]+"/"+c.env+".yaml", &cfg.AppConfigODS) 138 | } 139 | -------------------------------------------------------------------------------- /config/test.yaml: -------------------------------------------------------------------------------- 1 | log_type: zap 2 | 3 | logging: 4 | level: debug 5 | 6 | state_update_interval: 10s 7 | worker_idle_interval: 1s 8 | state_connect_url: "storagetapper:storagetapper@localhost" 9 | 10 | pipe: 11 | kafka: 12 | addresses: 13 | - "localhost:9091" 14 | - "localhost:9092" 15 | - "localhost:9093" 16 | hadoop: 17 | user: hadoop 18 | addresses: 19 | - "localhost:9000" 20 | base_dir: "/user/hadoop/tmp/hdfs_pipe_test" 21 | s3: 22 | region: "us-east-1" 23 | endpoint: "http://172.17.0.1:8000" 24 | access_key_id: 'accessKey1' 25 | secret_access_key: 'verySecretKey1' 26 | bucket: "pipetestbucket" 27 | base_dir: "/tmp/s3_pipe_test" 28 | 29 | output_topic_name_template: 30 | mysql: 31 | mysql: 32 | "{{.DB}}.{{.Table}}" 33 | -------------------------------------------------------------------------------- /config/topic.go: -------------------------------------------------------------------------------- 1 | // Copyright (c) 2017 Uber Technologies, Inc. 2 | // 3 | // Permission is hereby granted, free of charge, to any person obtaining a copy 4 | // of this software and associated documentation files (the "Software"), to deal 5 | // in the Software without restriction, including without limitation the rights 6 | // to use, copy, modify, merge, publish, distribute, sublicense, and/or sell 7 | // copies of the Software, and to permit persons to whom the Software is 8 | // furnished to do so, subject to the following conditions: 9 | // 10 | // The above copyright notice and this permission notice shall be included in 11 | // all copies or substantial portions of the Software. 12 | // 13 | // THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR 14 | // IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, 15 | // FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE 16 | // AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER 17 | // LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, 18 | // OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN 19 | // THE SOFTWARE. 20 | 21 | package config 22 | 23 | import ( 24 | "bytes" 25 | "text/template" 26 | "time" 27 | 28 | "github.com/uber/storagetapper/types" 29 | ) 30 | 31 | type topicParams struct { 32 | types.TableLoc 33 | Timestamp time.Time 34 | } 35 | 36 | func getTopicName(template *template.Template, tp *topicParams) (string, error) { 37 | buf := &bytes.Buffer{} 38 | err := template.Execute(buf, tp) 39 | if err != nil { 40 | return "", err 41 | } 42 | return buf.String(), nil 43 | } 44 | 45 | // GetOutputTopicName returns output topic name 46 | func (c *AppConfig) GetOutputTopicName(svc string, db string, tbl string, input string, output string, ver int, ts time.Time) (string, error) { 47 | tmpl := c.OutputTopicNameTemplateDefaultParsed 48 | 49 | inp := c.OutputTopicNameTemplateParsed[input] 50 | if inp != nil { 51 | out := inp[output] 52 | if out != nil { 53 | tmpl = out 54 | } 55 | } 56 | 57 | return getTopicName(tmpl, &topicParams{types.TableLoc{Service: svc, Cluster: "", DB: db, Table: tbl, Input: input, Output: output, Version: ver}, ts}) 58 | } 59 | 60 | // GetChangelogTopicName returns output topic name 61 | func (c *AppConfig) GetChangelogTopicName(svc string, db string, tbl string, input string, output string, ver int, ts time.Time) (string, error) { 62 | tmpl := c.ChangelogTopicNameTemplateDefaultParsed 63 | 64 | inp := c.ChangelogTopicNameTemplateParsed[input] 65 | if inp != nil { 66 | out := inp[output] 67 | if out != nil { 68 | tmpl = out 69 | } 70 | } 71 | 72 | return getTopicName(tmpl, &topicParams{types.TableLoc{Service: svc, Cluster: "", DB: db, Table: tbl, Input: input, Output: output, Version: ver}, ts}) 73 | } 74 | -------------------------------------------------------------------------------- /db/builtin.go: -------------------------------------------------------------------------------- 1 | // Copyright (c) 2017 Uber Technologies, Inc. 2 | // 3 | // Permission is hereby granted, free of charge, to any person obtaining a copy 4 | // of this software and associated documentation files (the "Software"), to deal 5 | // in the Software without restriction, including without limitation the rights 6 | // to use, copy, modify, merge, publish, distribute, sublicense, and/or sell 7 | // copies of the Software, and to permit persons to whom the Software is 8 | // furnished to do so, subject to the following conditions: 9 | // 10 | // The above copyright notice and this permission notice shall be included in 11 | // all copies or substantial portions of the Software. 12 | // 13 | // THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR 14 | // IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, 15 | // FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE 16 | // AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER 17 | // LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, 18 | // OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN 19 | // THE SOFTWARE. 20 | 21 | package db 22 | 23 | import ( 24 | "context" 25 | "fmt" 26 | 27 | "github.com/pkg/errors" 28 | "github.com/uber/storagetapper/log" 29 | "github.com/uber/storagetapper/types" 30 | ) 31 | 32 | func init() { 33 | registerResolver(types.InputMySQL, NewBuiltinResolver) 34 | } 35 | 36 | //BuiltinResolveCluster to implement runtime polymorphism and resolve cyclic 37 | //package dependency. state.SchemaGet will be assigned to in main.go and 38 | //connectInfoGetFortest in db/open_test.go 39 | var BuiltinResolveCluster func(*Loc, ConnectionType) (*Addr, error) 40 | 41 | // BuiltinResolver resolves MySQL connection info for SOA databases using Nemo 42 | type BuiltinResolver struct { 43 | } 44 | 45 | // NewBuiltinResolver returns a new resolver for resolving connection info using Nemo 46 | func NewBuiltinResolver() Resolver { 47 | return &BuiltinResolver{} 48 | } 49 | 50 | // GetInfo returns MySQL connection info 51 | func (r *BuiltinResolver) GetInfo(ctx context.Context, dbl *Loc, connType ConnectionType) (*Addr, error) { 52 | 53 | if BuiltinResolveCluster != nil { 54 | return BuiltinResolveCluster(dbl, connType) 55 | } 56 | 57 | return nil, fmt.Errorf("no builtin resolver set") 58 | } 59 | 60 | // IsValidConn checks the validity of the connection to make sure connection is to the correct MySQL DB 61 | func (r *BuiltinResolver) IsValidConn(ctx context.Context, dbl *Loc, connType ConnectionType, addr *Addr) bool { 62 | l := log.WithFields(log.Fields{"service": dbl.Service, "cluster": dbl.Cluster, "db": dbl.Name, "host": addr.Host, "port": addr.Port}) 63 | 64 | conn, err := r.GetInfo(ctx, dbl, connType) 65 | if err != nil { 66 | l.Warnf(errors.Wrap(err, "Failed to fetch connection info to compare, assuming valid connection").Error()) 67 | return true 68 | } 69 | 70 | if conn.Host != addr.Host || conn.Port != addr.Port { 71 | l.Warnf("Invalid connection due to mismatched host, port") 72 | return false 73 | } 74 | 75 | return true 76 | } 77 | 78 | // BuiltinEnumerator implements a db location enumerator for MySQL 79 | type BuiltinEnumerator struct { 80 | current int 81 | data []*Loc 82 | } 83 | 84 | // GetEnumerator returns an enumerator for db location info fetched from Nemo 85 | func (r *BuiltinResolver) GetEnumerator(ctx context.Context, svc, cluster, sdb, table string) (Enumerator, error) { 86 | if svc == "" || sdb == "" || table == "" || cluster == "" { 87 | err := errors.New("service name, cluster name, DB name or table name not provided") 88 | return nil, err 89 | } 90 | 91 | loc := Loc{Cluster: cluster, Service: svc, Name: sdb} 92 | return &BuiltinEnumerator{ 93 | current: -1, data: []*Loc{&loc}, 94 | }, nil 95 | } 96 | 97 | // Next is used to check if anymore values can be returned by the enumerator 98 | func (e *BuiltinEnumerator) Next() bool { 99 | e.current++ 100 | 101 | return e.current < len(e.data) 102 | } 103 | 104 | // Value returns the current value from the enumerator 105 | func (e *BuiltinEnumerator) Value() *Loc { 106 | return e.data[e.current] 107 | } 108 | 109 | // Reset resets the enumerator to the starting point 110 | func (e *BuiltinEnumerator) Reset() { 111 | e.current = -1 112 | } 113 | -------------------------------------------------------------------------------- /db/gtid.go: -------------------------------------------------------------------------------- 1 | package db 2 | 3 | import ( 4 | "database/sql" 5 | 6 | "github.com/pkg/errors" 7 | "github.com/uber/storagetapper/log" 8 | ) 9 | 10 | // GetCurrentGTID returns current gtid set for the specified db address (host,port,user,password) 11 | func GetCurrentGTID(addr *Addr) (string, error) { 12 | var d *sql.DB 13 | var gtid string 14 | var err error 15 | 16 | if d, err = Open(addr); err == nil { 17 | err = d.QueryRow("SELECT @@global.gtid_executed").Scan(>id) 18 | log.E(d.Close()) 19 | } 20 | 21 | return gtid, err 22 | } 23 | 24 | // GetPurgedGTID returns purged gtid set for the specified db address (host,port,user,password) 25 | func GetPurgedGTID(addr *Addr) (string, error) { 26 | var d *sql.DB 27 | var gtid string 28 | var err error 29 | 30 | if d, err = Open(addr); err == nil { 31 | err = d.QueryRow("SELECT @@global.gtid_purged").Scan(>id) 32 | log.E(d.Close()) 33 | } 34 | 35 | return gtid, err 36 | } 37 | 38 | // GetCurrentGTIDForDB return current gtid set for the db specified by db locator (cluster,service,db) 39 | func GetCurrentGTIDForDB(loc *Loc, inputType string) (string, error) { 40 | var err error 41 | var addr *Addr 42 | 43 | if addr, err = GetConnInfo(loc, Slave, inputType); err == nil { 44 | return GetCurrentGTID(addr) 45 | } 46 | 47 | return "", errors.Wrap(err, "error resolving db info") 48 | } 49 | -------------------------------------------------------------------------------- /db/gtid_test.go: -------------------------------------------------------------------------------- 1 | package db 2 | 3 | import ( 4 | "testing" 5 | 6 | "github.com/stretchr/testify/require" 7 | "github.com/uber/storagetapper/types" 8 | ) 9 | 10 | func TestGetCurrentGTID(t *testing.T) { 11 | addr := &Addr{Host: "localhost", Port: 3306, User: types.TestMySQLUser, Pwd: types.TestMySQLPassword} 12 | d, err := Open(addr) 13 | if err != nil { 14 | t.Skip("No MySQL connection") 15 | } 16 | if err := d.Close(); err != nil { 17 | t.Fatalf("Close failed") 18 | } 19 | 20 | gtid, err := GetCurrentGTID(addr) 21 | require.NoError(t, err) 22 | require.NotEmpty(t, gtid) 23 | } 24 | 25 | func TestGetCurrentGTIDForDB(t *testing.T) { 26 | addr := &Addr{Host: "localhost", Port: 3306, User: types.TestMySQLUser, Pwd: types.TestMySQLPassword} 27 | d, err := Open(addr) 28 | if err != nil { 29 | t.Skip("No MySQL connection") 30 | } 31 | if err := d.Close(); err != nil { 32 | t.Fatalf("Close failed") 33 | } 34 | 35 | //FIXME: Set correct user and password so the it can connect to MySQL and 36 | //return correct gtid set 37 | gtid, err := GetCurrentGTIDForDB(&Loc{Service: "test_svc1", Cluster: "test_cluster3", Name: "test_db2"}, "mysql") 38 | require.Error(t, err) 39 | require.Empty(t, gtid) 40 | } 41 | -------------------------------------------------------------------------------- /db/open_test.go: -------------------------------------------------------------------------------- 1 | // Copyright (c) 2017 Uber Technologies, Inc. 2 | // 3 | // Permission is hereby granted, free of charge, to any person obtaining a copy 4 | // of this software and associated documentation files (the "Software"), to deal 5 | // in the Software without restriction, including without limitation the rights 6 | // to use, copy, modify, merge, publish, distribute, sublicense, and/or sell 7 | // copies of the Software, and to permit persons to whom the Software is 8 | // furnished to do so, subject to the following conditions: 9 | // 10 | // The above copyright notice and this permission notice shall be included in 11 | // all copies or substantial portions of the Software. 12 | // 13 | // THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR 14 | // IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, 15 | // FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE 16 | // AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER 17 | // LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, 18 | // OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN 19 | // THE SOFTWARE. 20 | 21 | package db 22 | 23 | import ( 24 | "os" 25 | "testing" 26 | 27 | "github.com/uber/storagetapper/config" 28 | "github.com/uber/storagetapper/log" 29 | "github.com/uber/storagetapper/metrics" 30 | "github.com/uber/storagetapper/types" 31 | 32 | "github.com/stretchr/testify/require" 33 | ) 34 | 35 | var cfg *config.AppConfig 36 | 37 | func connectInfoGetForTest(l *Loc, connType ConnectionType) (*Addr, error) { 38 | return &Addr{Host: "localhost", Port: 3306, User: types.TestMySQLUser, Pwd: types.TestMySQLPassword, DB: l.Name}, nil 39 | } 40 | 41 | func TestBasic(t *testing.T) { 42 | d, err := Open(&Addr{Host: "localhost", Port: 3306, User: types.TestMySQLUser, Pwd: types.TestMySQLPassword}) 43 | if err != nil { 44 | t.Skip("No MySQL connection") 45 | } 46 | if err := d.Close(); err != nil { 47 | t.Fatalf("Close failed") 48 | } 49 | 50 | _, err = Open(&Addr{Host: "localhost", Port: 3307, User: "root"}) 51 | if err == nil { 52 | t.Fatalf("Open not failed") 53 | } 54 | 55 | _, err = OpenService(&Loc{Service: "test_svc1", Name: "test_db1"}, "", types.InputMySQL) 56 | if err == nil { 57 | t.Fatalf("OpenService not failed") 58 | } 59 | 60 | _, err = OpenService(&Loc{Service: "test_svc1", Name: "test_db2"}, "subst", types.InputMySQL) 61 | if err == nil { 62 | t.Fatalf("OpenService not failed") 63 | } 64 | } 65 | 66 | func TestGetEnumerator(t *testing.T) { 67 | // First test that GetEnumerator fails when service name is not provided 68 | _, err := GetEnumerator("", "storagetapper", "test_db2", "test_table1", "mysql") 69 | require.Error(t, err) 70 | 71 | // Next we check that GetEnumerator fails for invalid input type 72 | _, err = GetEnumerator("test_svc1", "test_cluster1", "test_db2", "test_table1", "unknown") 73 | require.Error(t, err) 74 | 75 | // Let's check correct flow now 76 | e, err := GetEnumerator("test_svc1", "test_cluster1", "test_db2", "test_table1", "mysql") 77 | require.NoError(t, err) 78 | require.NotNil(t, e) 79 | 80 | // Test that enumerator returns expected number of items 81 | var locs []*Loc 82 | for e.Next() { 83 | loc := e.Value() 84 | locs = append(locs, loc) 85 | require.Equal(t, "test_svc1", loc.Service) 86 | require.Equal(t, "test_db2", loc.Name) 87 | } 88 | 89 | require.Equal(t, 1, len(locs)) 90 | 91 | // Now test resetting of enumerator 92 | e.Reset() 93 | for e.Next() { 94 | loc := e.Value() 95 | require.Equal(t, "test_svc1", loc.Service) 96 | require.Equal(t, "test_db2", loc.Name) 97 | } 98 | 99 | //Constructing with empty cluster name 100 | e, err = GetEnumerator("test_svc1", "test_clst1", "test_db2", "test_table1", "mysql") 101 | require.NoError(t, err) 102 | require.NotNil(t, e) 103 | 104 | locs = make([]*Loc, 0) 105 | for e.Next() { 106 | loc := e.Value() 107 | locs = append(locs, loc) 108 | require.Equal(t, "test_svc1", loc.Service) 109 | require.Equal(t, "test_db2", loc.Name) 110 | } 111 | 112 | require.Equal(t, 1, len(locs)) 113 | } 114 | 115 | func TestMain(m *testing.M) { 116 | cfg = config.Get() 117 | 118 | log.Configure(cfg.LogType, cfg.LogLevel, config.Environment() == config.EnvProduction) 119 | 120 | BuiltinResolveCluster = connectInfoGetForTest 121 | 122 | err := metrics.Init() 123 | log.F(err) 124 | 125 | os.Exit(m.Run()) 126 | } 127 | -------------------------------------------------------------------------------- /db/resolver.go: -------------------------------------------------------------------------------- 1 | // Copyright (c) 2017 Uber Technologies, Inc. 2 | // 3 | // Permission is hereby granted, free of charge, to any person obtaining a copy 4 | // of this software and associated documentation files (the "Software"), to deal 5 | // in the Software without restriction, including without limitation the rights 6 | // to use, copy, modify, merge, publish, distribute, sublicense, and/or sell 7 | // copies of the Software, and to permit persons to whom the Software is 8 | // furnished to do so, subject to the following conditions: 9 | // 10 | // The above copyright notice and this permission notice shall be included in 11 | // all copies or substantial portions of the Software. 12 | // 13 | // THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR 14 | // IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, 15 | // FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE 16 | // AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER 17 | // LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, 18 | // OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN 19 | // THE SOFTWARE. 20 | 21 | package db 22 | 23 | import ( 24 | "context" 25 | 26 | "github.com/pkg/errors" 27 | 28 | "github.com/uber/storagetapper/log" 29 | ) 30 | 31 | // ConnectionType is used as enum to represent connection types 32 | type ConnectionType int 33 | 34 | const ( 35 | // Master is the type corresponding to a master or a leader 36 | Master ConnectionType = iota 37 | 38 | // Slave is the type corresponding to a slave or a follower 39 | Slave 40 | ) 41 | 42 | func (c ConnectionType) String() string { 43 | return [...]string{"master", "slave"}[c] 44 | } 45 | 46 | type resolverConstructor func() Resolver 47 | 48 | // resolvers is a registry of resolver initializers 49 | var resolvers map[string]resolverConstructor 50 | 51 | // Addr contains information required to connect to DB 52 | type Addr struct { 53 | Name string `json:"name"` 54 | Host string `json:"host"` 55 | Port uint16 `json:"port"` 56 | User string `json:"user"` 57 | Pwd string `json:"pwd"` 58 | DB string `json:"db"` 59 | } 60 | 61 | // Loc contains information related to database location 62 | type Loc struct { 63 | Cluster string 64 | Service string 65 | Name string 66 | } 67 | 68 | /*TableLoc - table location */ 69 | type TableLoc struct { 70 | Service string 71 | Cluster string 72 | DB string 73 | Table string 74 | Input string 75 | Output string 76 | Version int 77 | } 78 | 79 | // LogFields provides a logger that logs messages with Loc fields 80 | func (d *Loc) LogFields() log.Logger { 81 | return log.WithFields(log.Fields{ 82 | "cluster": d.Cluster, 83 | "service": d.Service, 84 | "db": d.Name, 85 | }) 86 | } 87 | 88 | // Resolver defines a MySQL connection info resolver 89 | type Resolver interface { 90 | GetInfo(context.Context, *Loc, ConnectionType) (*Addr, error) 91 | GetEnumerator(context.Context, string, string, string, string) (Enumerator, error) 92 | IsValidConn(ctx context.Context, dbl *Loc, connType ConnectionType, addr *Addr) bool 93 | } 94 | 95 | // NewResolver returns a MySQL connection info resolver depending on the input type 96 | func NewResolver(inputType string) (Resolver, error) { 97 | // If a resolver for the specified input type exists we create it 98 | if constructor, ok := resolvers[inputType]; ok { 99 | return constructor(), nil 100 | } 101 | 102 | return nil, errors.Errorf("failed to create resolver due to unknown input type %v", inputType) 103 | } 104 | 105 | // Enumerator defines an iterator that allows iterating over db locations 106 | type Enumerator interface { 107 | Value() *Loc 108 | Next() bool 109 | Reset() 110 | } 111 | 112 | // registerResolver is called by a specific resolver type to register itself 113 | func registerResolver(inputType string, c resolverConstructor) { 114 | if resolvers == nil { 115 | resolvers = make(map[string]resolverConstructor) 116 | } 117 | 118 | resolvers[inputType] = c 119 | } 120 | -------------------------------------------------------------------------------- /db/test.go: -------------------------------------------------------------------------------- 1 | // Copyright (c) 2017 Uber Technologies, Inc. 2 | // 3 | // Permission is hereby granted, free of charge, to any person obtaining a copy 4 | // of this software and associated documentation files (the "Software"), to deal 5 | // in the Software without restriction, including without limitation the rights 6 | // to use, copy, modify, merge, publish, distribute, sublicense, and/or sell 7 | // copies of the Software, and to permit persons to whom the Software is 8 | // furnished to do so, subject to the following conditions: 9 | // 10 | // The above copyright notice and this permission notice shall be included in 11 | // all copies or substantial portions of the Software. 12 | // 13 | // THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR 14 | // IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, 15 | // FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE 16 | // AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER 17 | // LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, 18 | // OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN 19 | // THE SOFTWARE. 20 | 21 | package db 22 | 23 | import ( 24 | "encoding/json" 25 | "fmt" 26 | "net/http" 27 | "net/http/httptest" 28 | 29 | "github.com/uber/storagetapper/log" 30 | "github.com/uber/storagetapper/types" 31 | ) 32 | 33 | // GetConnInfoForTest return address of local MySQL used by tests 34 | func GetConnInfoForTest(dbl *Loc, connType ConnectionType, inputType string) (*Addr, error) { 35 | log.Infof("Fetching connection info for tests: DB %v, connType %v, inputType %v", dbl.Name, connType, inputType) 36 | 37 | if dbl.Cluster == "please_return_nil_db_addr" { 38 | return nil, fmt.Errorf("here_you_are") 39 | } 40 | 41 | return &Addr{ 42 | Host: "localhost", 43 | Port: 3306, 44 | User: types.TestMySQLUser, 45 | Pwd: types.TestMySQLPassword, 46 | DB: dbl.Name, 47 | }, nil 48 | } 49 | 50 | // GetEnumeratorForTest return db location info enumerator used by tests 51 | func GetEnumeratorForTest(svc, cluster, sdb, table, inputType string) (Enumerator, error) { 52 | log.Infof("Fetching enumerator for tests: svc %v, db %v, table %v inputType %v", svc, sdb, table, 53 | inputType) 54 | 55 | loc := &Loc{ 56 | Cluster: cluster, 57 | Service: svc, 58 | Name: sdb, 59 | } 60 | 61 | return &BuiltinEnumerator{data: []*Loc{loc}, current: -1}, nil 62 | } 63 | 64 | // IsValidConnForTest is DB connection validator for tests 65 | func IsValidConnForTest(_ *Loc, _ ConnectionType, _ *Addr, _ string) bool { 66 | return true 67 | } 68 | 69 | // JSONServer starts a new server which serves json that's passed to it as an argument. 70 | // The string should be json with a map of url->response. If the response is "error" then 71 | // the server will return 500. 72 | func JSONServer(js string) (*httptest.Server, map[string]json.RawMessage) { 73 | var out map[string]json.RawMessage 74 | err := json.Unmarshal([]byte(js), &out) 75 | if err != nil { 76 | panic(err) 77 | } 78 | s := httptest.NewServer(http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) { 79 | e, ok := out[r.URL.Path] 80 | if !ok { 81 | e, ok = out[r.URL.String()] 82 | } 83 | if !ok { 84 | w.WriteHeader(404) 85 | if _, err := w.Write([]byte("Not found")); err != nil { 86 | panic(err) 87 | } 88 | return 89 | } 90 | b, err := e.MarshalJSON() 91 | if err != nil { 92 | panic(err) 93 | } 94 | if string(b) == "error" { 95 | w.WriteHeader(500) 96 | } 97 | if _, err := w.Write(b); err != nil { 98 | panic(err) 99 | } 100 | })) 101 | 102 | return s, out 103 | } 104 | -------------------------------------------------------------------------------- /debian/changelog: -------------------------------------------------------------------------------- 1 | storagetapper (1.0.0-beta) unstable; urgency=medium 2 | 3 | * Versioning scheme changed 4 | * Separate debian packages for server and client 5 | 6 | -- Evgeniy Firsov Tue, 15 Sep 2019 02:55:00 +0000 7 | 8 | storagetapper (1.0) unstable; urgency=medium 9 | 10 | * Initial version 11 | 12 | -- Evgeniy Firsov Tue, 3 May 2017 08:08:53 +0000 13 | -------------------------------------------------------------------------------- /debian/compat: -------------------------------------------------------------------------------- 1 | 9 2 | -------------------------------------------------------------------------------- /debian/control: -------------------------------------------------------------------------------- 1 | Source: storagetapper 2 | Priority: optional 3 | Maintainer: Evgeniy Firsov 4 | Uploaders: Evgeniy Firsov , 5 | Build-Depends: git, debhelper (>= 9), dh-systemd 6 | Standards-Version: 3.9.8 7 | Section: utils 8 | Homepage: https://github.com/uber/storagetapper 9 | 10 | Package: storagetapper-server 11 | Architecture: any 12 | Section: utils 13 | Depends: ${misc:Depends} 14 | Description: StorageTapper is a realtime change data capture and transformation 15 | service. This package contains server components. 16 | 17 | Package: storagetapper-client 18 | Architecture: any 19 | Section: utils 20 | Depends: ${misc:Depends} 21 | Description: StorageTapper is a realtime change data capture and transformation 22 | service. This package contains client components. 23 | 24 | 25 | -------------------------------------------------------------------------------- /debian/copyright: -------------------------------------------------------------------------------- 1 | Format: https:www.debian.org/doc/packaging-manuals/copyright-format/1.0/ 2 | Upstream-Name: Storagetapper 3 | Source: https:github.com/uber/storagetapper 4 | 5 | Files: * 6 | Copyright: 2017 Uber Technologies, Inc. 7 | License: Expat 8 | Permission is hereby granted, free of charge, to any person obtaining a copy 9 | of this software and associated documentation files (the "Software"), to deal 10 | in the Software without restriction, including without limitation the rights 11 | to use, copy, modify, merge, publish, distribute, sublicense, and/or sell 12 | copies of the Software, and to permit persons to whom the Software is 13 | furnished to do so, subject to the following conditions: 14 | 15 | The above copyright notice and this permission notice shall be included in 16 | all copies or substantial portions of the Software. 17 | 18 | THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR 19 | IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, 20 | FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE 21 | AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER 22 | LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, 23 | OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN 24 | THE SOFTWARE. 25 | 26 | -------------------------------------------------------------------------------- /debian/rules: -------------------------------------------------------------------------------- 1 | #!/usr/bin/make -f 2 | 3 | export DH_VERBOSE=1 4 | 5 | export DEB_BUILD_MAINT_OPTIONS=hardening=+all 6 | 7 | export PREFIX=/usr 8 | 9 | %: 10 | dh $@ --parallel --with systemd 11 | -------------------------------------------------------------------------------- /debian/source/format: -------------------------------------------------------------------------------- 1 | 3.0 (native) 2 | -------------------------------------------------------------------------------- /debian/storagetapper-client.install: -------------------------------------------------------------------------------- 1 | scripts/stcli usr/bin 2 | -------------------------------------------------------------------------------- /debian/storagetapper-server.install: -------------------------------------------------------------------------------- 1 | storagetapper usr/bin 2 | config/base.yaml etc/storagetapper 3 | config/production.yaml etc/storagetapper 4 | -------------------------------------------------------------------------------- /debian/storagetapper-server.postinst: -------------------------------------------------------------------------------- 1 | #!/bin/sh 2 | 3 | set -e 4 | 5 | if [ "$1" = "configure" ]; then 6 | if ! getent passwd storagetapper >/dev/null; then 7 | adduser --system --quiet --home / --no-create-home \ 8 | --shell /bin/false --group --gecos "storagetapper daemon user" storagetapper 9 | fi 10 | 11 | # Ensure configuration files are readable by the dedicated storagetapper user. 12 | chgrp -R storagetapper /etc/storagetapper 13 | 14 | # For upgrades: Ensure configuration files are not world-readable. 15 | find /etc/storagetapper -perm -o+r -type f -exec chmod 640 '{}' \; 16 | fi 17 | 18 | #DEBHELPER# 19 | 20 | exit 0 21 | -------------------------------------------------------------------------------- /debian/storagetapper-server.service: -------------------------------------------------------------------------------- 1 | [Unit] 2 | Description=StorageTapper realtime MySQL change data streaming 3 | 4 | [Service] 5 | User=storagetapper 6 | ExecStart=/usr/bin/storagetapper 7 | 8 | [Install] 9 | WantedBy=multi-user.target 10 | -------------------------------------------------------------------------------- /doc/commonformat.md: -------------------------------------------------------------------------------- 1 | Common JSON format 2 | ------------------ 3 | 4 | The intention is to define format general enough to transit modification events of different databases. 5 | 6 | CommonFormatEvent structure has the following fields: 7 | 8 | * Type - type of event: insert, delete, schema. updates is produces as delete/insert pair 9 | * Key - primary key of the row, encoded as an array 10 | * SeqNo - event sequence number generated by the reader 11 | * Timestamp - timestamp of the moment when event generated by the reader 12 | * Fields - Array of name/value pairs, empty for delete event 13 | 14 | Import [types/format.go](../types/format.go) in order to unmarshal events in Golang. 15 | 16 | ## Examples of events: 17 | 18 | ### Insert event: 19 | ```json 20 | { "Type":"insert", 21 | "Key":["part1","part2"], 22 | "SeqNo":123, 23 | "Timestamp":1494315140, 24 | "Fields": [ 25 | {"Name":"f1","Value":1}, 26 | {"Name":"f3","Value":"string field"}, 27 | {"Name":"f4","Value":null} 28 | ] 29 | } 30 | ``` 31 | ### Delete event: 32 | ```json 33 | { "Type":"delete", "Key":["key1"], "SeqNo":124, "Timestamp":1494315140} 34 | ``` 35 | 36 | ### Schema event: 37 | ```json 38 | {"Type":"schema","Key":["f1"],"SeqNo":125,"Timestamp":1494315140,"Fields":[{"Name":"f1","Value":"int(11)"},{"Name":"f3","Value":"int(11)"},{"Name":"f4","Value":"int(11)"}]} 39 | ``` 40 | -------------------------------------------------------------------------------- /doc/endpoints.md: -------------------------------------------------------------------------------- 1 | HTTP endpoints 2 | -------------- 3 | 4 | ## Builtin database resolver 5 | 6 | This provides source MySQL cluster connection information mapping to unique 7 | cluster name. 8 | 9 | http://localhost:7836/cluster 10 | 11 | ```json 12 | {"cmd" : "add", "name" : "cluster1", "host" : "localhost", "port" : 3306, "user" : "root", "pw" : ""}, 13 | {"cmd" : "del", "name" : "cluster1"} 14 | ``` 15 | 16 | ## Control tables for ingestion 17 | 18 | This endpoint controls table ingestion tasks. 19 | Note: "service" field is deprecated. It was used as top level scope allowing 20 | non-unique cluster names. 21 | 22 | "http://localhost:7836/table" 23 | 24 | ```json 25 | {"cmd" : "add", "cluster" : "cluster1", "service" : "service1", "db":"database1", "table":"table1"}, 26 | {"cmd" : "del", "cluster" : "cluster1", "service" : "service1", "db":"database1", "table":"table1"}, 27 | {"cmd" : "list", "cluster" : "cluster1", "service" : "service1", "db":"database1", "table":"table1"} 28 | ``` 29 | There is a helper CLI script, which provides convenient way to control table 30 | onboarding and offboarding [stcli](../scripts/stcli) 31 | 32 | ## Output schema store 33 | 34 | Schema is only required for Avro format, for JSON and MsgPack it's still can be 35 | added for field filtering for example. Fields which are ommitted in the schema, 36 | won't be produce to destination. 37 | 38 | http://localhost:7836/schema 39 | 40 | ```json 41 | {"cmd" : "add", "name" : "name1", "schema": "schema body in Avro or JSON format"} 42 | {"cmd" : "del", "name" : "name1"} 43 | {"cmd" : "register", "service" : "service1", "db":"database1", "table":"table1"} 44 | ``` 45 | -------------------------------------------------------------------------------- /docker_test_entry.sh: -------------------------------------------------------------------------------- 1 | #!/bin/bash 2 | 3 | set -ex 4 | 5 | export GOPATH=$HOME/gopath 6 | export PATH=$HOME/gopath/bin:/usr/local/go/bin:$PATH 7 | export UBER_DATACENTER="sjc1" 8 | 9 | /bin/sh scripts/prepare_test_env.sh 10 | 11 | P=$GOPATH/src/github.com/uber 12 | mkdir -p $P 13 | cp -ap /storagetapper $P 14 | cd $P/storagetapper 15 | 16 | PKGS=$(find . -maxdepth 1 -type d -not -path '*/\.*'|grep -v -e vendor -e doc -e debian -e scripts -e udeploy -e go-build -e idl -e testdata -e dashboard|sort -r) 17 | 18 | sh scripts/run_lints.sh $PKGS 19 | 20 | make -k test-xml GO_VERSION_SETUP_DISABLED=1 RACE="-race" TEST_VERBOSITY_FLAG="-v" PHAB_COMMENT=.phabricator-comment && cp coverage.xml junit.xml /storagetapper && chmod a+rw /storagetapper/coverage.xml /storagetapper/junit.xml 21 | -------------------------------------------------------------------------------- /encoder/avro_schema.go: -------------------------------------------------------------------------------- 1 | // Copyright (c) 2017 Uber Technologies, Inc. 2 | // 3 | // Permission is hereby granted, free of charge, to any person obtaining a copy 4 | // of this software and associated documentation files (the "Software"), to deal 5 | // in the Software without restriction, including without limitation the rights 6 | // to use, copy, modify, merge, publish, distribute, sublicense, and/or sell 7 | // copies of the Software, and to permit persons to whom the Software is 8 | // furnished to do so, subject to the following conditions: 9 | // 10 | // The above copyright notice and this permission notice shall be included in 11 | // all copies or substantial portions of the Software. 12 | // 13 | // THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR 14 | // IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, 15 | // FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE 16 | // AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER 17 | // LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, 18 | // OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN 19 | // THE SOFTWARE. 20 | 21 | package encoder 22 | 23 | import ( 24 | "encoding/json" 25 | "time" 26 | 27 | "github.com/linkedin/goavro" 28 | "github.com/uber/storagetapper/config" 29 | "github.com/uber/storagetapper/types" 30 | "github.com/uber/storagetapper/util" 31 | ) 32 | 33 | //GetLatestSchemaFunc is a type to implement schema resolver polymorphism 34 | type GetLatestSchemaFunc func(namespace string, schemaName string, typ string) (*types.AvroSchema, error) 35 | 36 | //GetLatestSchema is the pointer to schema resolver 37 | var GetLatestSchema GetLatestSchemaFunc 38 | 39 | //SchemaCodecHelper gets Avro codec and Avro record setter from schema structure 40 | func SchemaCodecHelper(avroSchema *types.AvroSchema) (goavro.Codec, *goavro.RecordSetter, error) { 41 | sch, err := json.Marshal(avroSchema) 42 | if err != nil { 43 | return nil, nil, err 44 | } 45 | schStr := util.BytesToString(sch) 46 | 47 | codec, err := goavro.NewCodec(schStr) 48 | if err != nil { 49 | return nil, nil, err 50 | } 51 | recSch := goavro.RecordSchema(schStr) 52 | return codec, &recSch, nil 53 | } 54 | 55 | //GetLatestSchemaCodec resolves schema and converts it to Avro codec and setter 56 | func GetLatestSchemaCodec(service string, db string, table string, typ string, input string, output string, version int) (goavro.Codec, *goavro.RecordSetter, error) { 57 | n, err := GetOutputSchemaName(service, db, table, input, output, version) 58 | if err != nil { 59 | return nil, nil, err 60 | } 61 | avroSchema, err := GetLatestSchema("production", n, typ) 62 | if err != nil { 63 | return nil, nil, err 64 | } 65 | return SchemaCodecHelper(avroSchema) 66 | } 67 | 68 | //GetOutputSchemaName combines parameter into output topic name 69 | func GetOutputSchemaName(service string, db string, table string, input string, output string, version int) (string, error) { 70 | return config.Get().GetOutputTopicName(service, db, table, input, output, version, time.Now()) 71 | } 72 | -------------------------------------------------------------------------------- /encoder/avro_schema_test.go: -------------------------------------------------------------------------------- 1 | // Copyright (c) 2017 Uber Technologies, Inc. 2 | // 3 | // Permission is hereby granted, free of charge, to any person obtaining a copy 4 | // of this software and associated documentation files (the "Software"), to deal 5 | // in the Software without restriction, including without limitation the rights 6 | // to use, copy, modify, merge, publish, distribute, sublicense, and/or sell 7 | // copies of the Software, and to permit persons to whom the Software is 8 | // furnished to do so, subject to the following conditions: 9 | // 10 | // The above copyright notice and this permission notice shall be included in 11 | // all copies or substantial portions of the Software. 12 | // 13 | // THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR 14 | // IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, 15 | // FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE 16 | // AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER 17 | // LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, 18 | // OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN 19 | // THE SOFTWARE. 20 | 21 | package encoder 22 | 23 | import ( 24 | "testing" 25 | 26 | "encoding/json" 27 | 28 | "github.com/uber/storagetapper/test" 29 | "github.com/uber/storagetapper/types" 30 | ) 31 | 32 | /* 33 | type inout struct { 34 | in []interface{} 35 | out string 36 | } 37 | 38 | func TestGetRowKeyFromPKey(t *testing.T) { 39 | tests := []inout{} 40 | tests = append(tests, inout{in: []interface{}{"abc"}, out: "3abc"}) 41 | tests = append(tests, inout{in: []interface{}{"abc", "def"}, out: "3abc3def"}) 42 | tests = append(tests, inout{in: []interface{}{"ab", "cdef"}, out: "2ab4cdef"}) 43 | tests = append(tests, inout{in: []interface{}{12, 345}, out: "2123345"}) 44 | tests = append(tests, inout{in: []interface{}{123, 45}, out: "3123245"}) 45 | tests = append(tests, inout{in: []interface{}{"a1b2", "3def"}, out: "4a1b243def"}) 46 | tests = append(tests, inout{in: []interface{}{"j4", "3.13245985"}, out: "2j4103.13245985"}) 47 | for _, test := range tests { 48 | test.Assert(t, GetRowKeyFromPKey(test.in), test.out) 49 | } 50 | } 51 | */ 52 | 53 | func TestGetLatestSchema(t *testing.T) { 54 | av1 := `{"owner":"abcd@example.com","fields":[{"default":null,"type":["null","boolean"],"name":"test"}],"namespace":"test","name":"TEST","type":"record"}` 55 | var avSch types.AvroSchema 56 | err := json.Unmarshal([]byte(av1), &avSch) 57 | test.Assert(t, err == nil, "Error unmarshalling latest Avro schema into AvroSchema object") 58 | } 59 | 60 | func TestSchemaCodecHelper(t *testing.T) { 61 | av1 := `{"owner":"abcd@example.com","fields":[{"default":null,"type":["null","boolean"],"name":"test"}],"namespace":"test","name":"TEST","type":"record"}` 62 | var avSch types.AvroSchema 63 | err := json.Unmarshal([]byte(av1), &avSch) 64 | test.Assert(t, err == nil, "Error unmarshalling latest Avro schema into AvroSchema object") 65 | _, _, err = SchemaCodecHelper(&avSch) 66 | test.Assert(t, err == nil, "Not OK") 67 | } 68 | -------------------------------------------------------------------------------- /encoder/z.go: -------------------------------------------------------------------------------- 1 | // Copyright (c) 2017 Uber Technologies, Inc. 2 | // 3 | // Permission is hereby granted, free of charge, to any person obtaining a copy 4 | // of this software and associated documentation files (the "Software"), to deal 5 | // in the Software without restriction, including without limitation the rights 6 | // to use, copy, modify, merge, publish, distribute, sublicense, and/or sell 7 | // copies of the Software, and to permit persons to whom the Software is 8 | // furnished to do so, subject to the following conditions: 9 | // 10 | // The above copyright notice and this permission notice shall be included in 11 | // all copies or substantial portions of the Software. 12 | // 13 | // THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR 14 | // IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, 15 | // FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE 16 | // AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER 17 | // LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, 18 | // OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN 19 | // THE SOFTWARE. 20 | 21 | package encoder 22 | 23 | import ( 24 | "fmt" 25 | 26 | "github.com/uber/storagetapper/config" 27 | ) 28 | 29 | //This init must be called after all calls to init in the package, allowing all encoders 30 | //to register, before we initializing internal encoder in here. 31 | //To ensure that it's called last the name of the file should be greater then 32 | //any other file in the package in lexical order. 33 | // 34 | //https://golang.org/ref/spec 35 | //To ensure reproducible initialization behavior, build systems are encouraged to present multiple files belonging to the same package in lexical file name order to a compiler. 36 | 37 | func init() { 38 | var err error 39 | Internal, err = InitEncoder(config.Get().InternalEncoding, "", "", "", "", "", 0) 40 | if err != nil { 41 | panic(fmt.Sprintf("Set InternalEncoding to json. Error: %s", err.Error())) 42 | } 43 | } 44 | -------------------------------------------------------------------------------- /go.mod: -------------------------------------------------------------------------------- 1 | module github.com/uber/storagetapper 2 | 3 | go 1.12 4 | 5 | require ( 6 | github.com/ProtonMail/go-crypto v0.0.0-20211112122917-428f8eabeeb3 7 | github.com/Shopify/sarama v1.30.0 8 | github.com/aws/aws-sdk-go v1.42.7 9 | github.com/cactus/go-statsd-client/statsd v0.0.0-20191106001114-12b4e2b38748 10 | github.com/davecgh/go-spew v1.1.1 11 | github.com/efirs/hdfs/v2 v2.2.1 12 | github.com/fsnotify/fsnotify v1.5.1 13 | github.com/go-mysql-org/go-mysql v1.3.0 14 | github.com/go-sql-driver/mysql v1.6.0 15 | github.com/gofrs/uuid v4.1.0+incompatible 16 | github.com/golang/protobuf v1.5.2 // indirect 17 | github.com/google/uuid v1.3.0 // indirect 18 | github.com/lib/pq v1.10.4 19 | github.com/linkedin/goavro v1.0.5 20 | github.com/mailru/go-clickhouse v1.7.0 21 | github.com/pkg/errors v0.9.1 22 | github.com/shopspring/decimal v1.3.1 // indirect 23 | github.com/siddontang/go-log v0.0.0-20190221022429-1e957dd83bed // indirect 24 | github.com/sirupsen/logrus v1.8.1 25 | github.com/stretchr/objx v0.2.0 // indirect 26 | github.com/stretchr/testify v1.7.0 27 | github.com/tinylib/msgp v1.1.6 28 | github.com/twmb/murmur3 v1.1.6 // indirect 29 | github.com/uber-common/bark v1.3.0 30 | github.com/uber-go/tally v3.4.2+incompatible 31 | go.uber.org/atomic v1.9.0 // indirect 32 | go.uber.org/multierr v1.7.0 // indirect 33 | go.uber.org/zap v1.19.1 34 | golang.org/x/crypto v0.0.0-20211117183948-ae814b36b871 // indirect 35 | golang.org/x/net v0.0.0-20211116231205-47ca1ff31462 36 | golang.org/x/sync v0.0.0-20210220032951-036812b2e83c 37 | golang.org/x/sys v0.0.0-20211117180635-dee7805ff2e1 38 | google.golang.org/protobuf v1.27.1 // indirect 39 | gopkg.in/yaml.v2 v2.4.0 40 | ) 41 | -------------------------------------------------------------------------------- /log/log_test.go: -------------------------------------------------------------------------------- 1 | // Copyright (c) 2017 Uber Technologies, Inc. 2 | // 3 | // Permission is hereby granted, free of charge, to any person obtaining a copy 4 | // of this software and associated documentation files (the "Software"), to deal 5 | // in the Software without restriction, including without limitation the rights 6 | // to use, copy, modify, merge, publish, distribute, sublicense, and/or sell 7 | // copies of the Software, and to permit persons to whom the Software is 8 | // furnished to do so, subject to the following conditions: 9 | // 10 | // The above copyright notice and this permission notice shall be included in 11 | // all copies or substantial portions of the Software. 12 | // 13 | // THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR 14 | // IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, 15 | // FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE 16 | // AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER 17 | // LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, 18 | // OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN 19 | // THE SOFTWARE. 20 | 21 | package log 22 | 23 | import ( 24 | "fmt" 25 | "testing" 26 | ) 27 | 28 | func TestBasic(t *testing.T) { 29 | Configure("std", "debug", false) 30 | 31 | Debugf("msg") 32 | Infof("msg") 33 | Warnf("msg") 34 | Errorf("msg") 35 | 36 | if !E(fmt.Errorf("Test error to pint")) { 37 | t.Fatalf("Should return true when error is not nil") 38 | } 39 | 40 | if E(nil) { 41 | t.Fatalf("Should return false when error is nil") 42 | } 43 | 44 | l := WithFields(Fields{"Test field 1": "val1", "Test field2": "val2"}) 45 | 46 | l.Debugf("msg with fields") 47 | l.Infof("msg with fields") 48 | l.Warnf("msg with fields") 49 | l.Errorf("msg with fields") 50 | 51 | if !EL(l, fmt.Errorf("Print error with fields")) { 52 | t.Fatalf("Should return true when error is not nil") 53 | } 54 | 55 | l = WithFields(Fields{}) 56 | 57 | l.Debugf("msg with empty fields") 58 | l.Infof("msg with empty fields") 59 | l.Warnf("msg with empty fields") 60 | l.Errorf("msg with empty fields") 61 | } 62 | 63 | func TestWrongLogger(t *testing.T) { 64 | Configure("non_existent_logger", "debug", false) 65 | if _, ok := def.(*stdlog); !ok { 66 | t.Fatalf("stdlog should be created by default") 67 | } 68 | 69 | } 70 | 71 | func TestWrongLevel(t *testing.T) { 72 | if ParseLevel("non_existent_level") != Info { 73 | t.Fatalf("default log level should info") 74 | } 75 | } 76 | 77 | //implementation of log interface to test panicf and fatalf calls 78 | type nillog struct { 79 | fatalf int 80 | panicf int 81 | } 82 | 83 | //Debugf logs a message with given format and arguments 84 | func (log *nillog) Debugf(format string, args ...interface{}) { 85 | } 86 | 87 | //Infof logs a message with given format and arguments 88 | func (log *nillog) Infof(format string, args ...interface{}) { 89 | } 90 | 91 | //Warnf logs a message with given format and arguments 92 | func (log *nillog) Warnf(format string, args ...interface{}) { 93 | } 94 | 95 | //Errorf logs a message with given format and arguments 96 | func (log *nillog) Errorf(format string, args ...interface{}) { 97 | } 98 | 99 | //Fatalf logs a message with given format and arguments and then finishes the 100 | //process with error exit code 101 | func (log *nillog) Fatalf(format string, args ...interface{}) { 102 | log.fatalf++ 103 | } 104 | 105 | //Panicf logs a message with given format and arguments and then panics 106 | func (log *nillog) Panicf(format string, args ...interface{}) { 107 | log.panicf++ 108 | } 109 | 110 | //WithFields attaches given key/value fields and return new logger with those 111 | //fields attached 112 | func (log *nillog) WithFields(fields Fields) Logger { 113 | return nil 114 | } 115 | 116 | func configureNil(level int, production bool) (Logger, error) { 117 | return &nillog{}, nil 118 | } 119 | 120 | func TestPanicAndFatal(t *testing.T) { 121 | registerPlugin("nillog", configureNil) 122 | 123 | Configure("nillog", "debug", false) 124 | 125 | nl, ok := def.(*nillog) 126 | 127 | if !ok { 128 | t.Fatalf("nillog should be created and be default logger") 129 | } 130 | 131 | Panicf("msg") 132 | 133 | if nl.panicf != 1 { 134 | t.Fatalf("panicf wasn't called") 135 | } 136 | 137 | Fatalf("msg") 138 | 139 | if nl.fatalf != 1 { 140 | t.Fatalf("fatalf wasn't called") 141 | } 142 | 143 | F(fmt.Errorf("Test error to pint")) 144 | F(nil) 145 | 146 | if nl.fatalf != 2 { 147 | t.Fatalf("fatalf wasn't called") 148 | } 149 | 150 | } 151 | -------------------------------------------------------------------------------- /log/rus.go: -------------------------------------------------------------------------------- 1 | // Copyright (c) 2017 Uber Technologies, Inc. 2 | // 3 | // Permission is hereby granted, free of charge, to any person obtaining a copy 4 | // of this software and associated documentation files (the "Software"), to deal 5 | // in the Software without restriction, including without limitation the rights 6 | // to use, copy, modify, merge, publish, distribute, sublicense, and/or sell 7 | // copies of the Software, and to permit persons to whom the Software is 8 | // furnished to do so, subject to the following conditions: 9 | // 10 | // The above copyright notice and this permission notice shall be included in 11 | // all copies or substantial portions of the Software. 12 | // 13 | // THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR 14 | // IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, 15 | // FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE 16 | // AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER 17 | // LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, 18 | // OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN 19 | // THE SOFTWARE. 20 | 21 | package log 22 | 23 | import ( 24 | "github.com/sirupsen/logrus" 25 | "github.com/uber-common/bark" 26 | ) 27 | 28 | func init() { 29 | registerPlugin("logrus", configureLogrus) 30 | } 31 | 32 | //The indirection is needed to satisfy our logger's interface WithFields method 33 | type logrusLogger struct { 34 | bark.Logger 35 | } 36 | 37 | func configureLogrus(level int, production bool) (Logger, error) { 38 | l := logrus.New() 39 | l.Level = logrus.Level(level) 40 | if production { 41 | l.Formatter = new(logrus.JSONFormatter) 42 | } 43 | return &logrusLogger{bark.NewLoggerFromLogrus(l)}, nil 44 | } 45 | 46 | func (l *logrusLogger) WithFields(keyValues Fields) Logger { 47 | return &logrusLogger{l.Logger.WithFields(bark.Fields(keyValues))} 48 | } 49 | -------------------------------------------------------------------------------- /log/rus_test.go: -------------------------------------------------------------------------------- 1 | // Copyright (c) 2017 Uber Technologies, Inc. 2 | // 3 | // Permission is hereby granted, free of charge, to any person obtaining a copy 4 | // of this software and associated documentation files (the "Software"), to deal 5 | // in the Software without restriction, including without limitation the rights 6 | // to use, copy, modify, merge, publish, distribute, sublicense, and/or sell 7 | // copies of the Software, and to permit persons to whom the Software is 8 | // furnished to do so, subject to the following conditions: 9 | // 10 | // The above copyright notice and this permission notice shall be included in 11 | // all copies or substantial portions of the Software. 12 | // 13 | // THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR 14 | // IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, 15 | // FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE 16 | // AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER 17 | // LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, 18 | // OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN 19 | // THE SOFTWARE. 20 | 21 | package log 22 | 23 | import ( 24 | "fmt" 25 | "testing" 26 | ) 27 | 28 | func TestLogrusBasic(t *testing.T) { 29 | for i := 0; i <= Debug; i++ { 30 | fmt.Printf("Level: %v\n", i) 31 | 32 | l, err := configureLogrus(i, false) 33 | if err != nil { 34 | t.Fatalf("Init failed %v", err.Error()) 35 | } 36 | 37 | l.Debugf("msg") 38 | l.Infof("msg") 39 | l.Warnf("msg") 40 | l.Errorf("msg") 41 | 42 | lf := l.WithFields(Fields{"Test field 1": "val1", "Test field2": "val2"}) 43 | 44 | lf.Debugf("msg with fields") 45 | lf.Infof("msg with fields") 46 | lf.Warnf("msg with fields") 47 | lf.Errorf("msg with fields") 48 | } 49 | } 50 | -------------------------------------------------------------------------------- /log/std.go: -------------------------------------------------------------------------------- 1 | // Copyright (c) 2017 Uber Technologies, Inc. 2 | // 3 | // Permission is hereby granted, free of charge, to any person obtaining a copy 4 | // of this software and associated documentation files (the "Software"), to deal 5 | // in the Software without restriction, including without limitation the rights 6 | // to use, copy, modify, merge, publish, distribute, sublicense, and/or sell 7 | // copies of the Software, and to permit persons to whom the Software is 8 | // furnished to do so, subject to the following conditions: 9 | // 10 | // The above copyright notice and this permission notice shall be included in 11 | // all copies or substantial portions of the Software. 12 | // 13 | // THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR 14 | // IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, 15 | // FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE 16 | // AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER 17 | // LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, 18 | // OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN 19 | // THE SOFTWARE. 20 | 21 | package log 22 | 23 | import ( 24 | "fmt" 25 | "log" 26 | "os" 27 | ) 28 | 29 | func init() { 30 | registerPlugin("std", configureStd) 31 | } 32 | 33 | func newStd(level int) Logger { 34 | return &stdlog{level: level, impl: log.New(os.Stderr, "", log.Lshortfile), skipFrames: 3} 35 | } 36 | 37 | func configureStd(level int, production bool) (Logger, error) { 38 | return newStd(level), nil 39 | } 40 | 41 | type stdlog struct { 42 | level int 43 | impl *log.Logger 44 | skipFrames int 45 | } 46 | 47 | func (log *stdlog) output(level int, format string, args ...interface{}) { 48 | if log.level >= level { 49 | _ = log.impl.Output(log.skipFrames+1, fmt.Sprintf(format, args...)) 50 | } 51 | } 52 | 53 | //Debugf logs a message with given format and arguments 54 | func (log *stdlog) Debugf(format string, args ...interface{}) { 55 | log.output(Debug, format, args...) 56 | } 57 | 58 | //Infof logs a message with given format and arguments 59 | func (log *stdlog) Infof(format string, args ...interface{}) { 60 | log.output(Info, format, args...) 61 | } 62 | 63 | //Warnf logs a message with given format and arguments 64 | func (log *stdlog) Warnf(format string, args ...interface{}) { 65 | log.output(Warn, format, args...) 66 | } 67 | 68 | //Errorf logs a message with given format and arguments 69 | func (log *stdlog) Errorf(format string, args ...interface{}) { 70 | log.output(Error, format, args...) 71 | } 72 | 73 | //Fatalf logs a message with given format and arguments and then finishes the 74 | //process with error exit code 75 | func (log *stdlog) Fatalf(format string, args ...interface{}) { 76 | log.output(Fatal, format, args...) 77 | os.Exit(1) 78 | } 79 | 80 | //Panicf logs a message with given format and arguments and then panics 81 | func (log *stdlog) Panicf(format string, args ...interface{}) { 82 | log.output(Panic, format, args...) 83 | panic(fmt.Sprintf(format, args...)) 84 | } 85 | 86 | //WithFields attaches given key/value fields and return new logger with those 87 | //fields attached 88 | func (log *stdlog) WithFields(fields Fields) Logger { 89 | n := newStd(log.level).(*stdlog) 90 | n.skipFrames = 2 91 | 92 | if len(fields) == 0 { 93 | return n 94 | } 95 | 96 | var s = log.impl.Prefix() 97 | for k, v := range fields { 98 | if len(s) > 0 { 99 | s += ", " 100 | } 101 | s += fmt.Sprintf("%v: %v", k, v) 102 | } 103 | 104 | n.impl.SetPrefix(s + ": ") 105 | 106 | return n 107 | } 108 | -------------------------------------------------------------------------------- /log/std_test.go: -------------------------------------------------------------------------------- 1 | // Copyright (c) 2017 Uber Technologies, Inc. 2 | // 3 | // Permission is hereby granted, free of charge, to any person obtaining a copy 4 | // of this software and associated documentation files (the "Software"), to deal 5 | // in the Software without restriction, including without limitation the rights 6 | // to use, copy, modify, merge, publish, distribute, sublicense, and/or sell 7 | // copies of the Software, and to permit persons to whom the Software is 8 | // furnished to do so, subject to the following conditions: 9 | // 10 | // The above copyright notice and this permission notice shall be included in 11 | // all copies or substantial portions of the Software. 12 | // 13 | // THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR 14 | // IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, 15 | // FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE 16 | // AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER 17 | // LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, 18 | // OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN 19 | // THE SOFTWARE. 20 | 21 | package log 22 | 23 | import ( 24 | "fmt" 25 | "testing" 26 | ) 27 | 28 | func TestStdBasic(t *testing.T) { 29 | for i := 0; i <= Debug; i++ { 30 | fmt.Printf("Level: %v\n", i) 31 | 32 | var err error 33 | def, err = configureStd(i, false) 34 | if err != nil { 35 | t.Fatalf("Init failed %v", err.Error()) 36 | } 37 | 38 | Debugf("msg") 39 | Infof("msg") 40 | Warnf("msg") 41 | Errorf("msg") 42 | 43 | lf := WithFields(Fields{"Test field 1": "val1", "Test field2": "val2"}) 44 | 45 | lf.Debugf("msg with fields") 46 | lf.Infof("msg with fields") 47 | lf.Warnf("msg with fields") 48 | lf.Errorf("msg with fields") 49 | 50 | Debugf("msg") 51 | Infof("msg") 52 | Warnf("msg") 53 | Errorf("msg") 54 | } 55 | } 56 | -------------------------------------------------------------------------------- /log/zap.go: -------------------------------------------------------------------------------- 1 | // Copyright (c) 2017 Uber Technologies, Inc. 2 | // 3 | // Permission is hereby granted, free of charge, to any person obtaining a copy 4 | // of this software and associated documentation files (the "Software"), to deal 5 | // in the Software without restriction, including without limitation the rights 6 | // to use, copy, modify, merge, publish, distribute, sublicense, and/or sell 7 | // copies of the Software, and to permit persons to whom the Software is 8 | // furnished to do so, subject to the following conditions: 9 | // 10 | // The above copyright notice and this permission notice shall be included in 11 | // all copies or substantial portions of the Software. 12 | // 13 | // THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR 14 | // IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, 15 | // FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE 16 | // AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER 17 | // LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, 18 | // OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN 19 | // THE SOFTWARE. 20 | 21 | //go:build go1.7 22 | // +build go1.7 23 | 24 | package log 25 | 26 | import ( 27 | "go.uber.org/zap" 28 | "go.uber.org/zap/zapcore" 29 | 30 | "github.com/uber/storagetapper/config" 31 | ) 32 | 33 | func init() { 34 | registerPlugin("zap", configureZap) 35 | } 36 | 37 | type zapConfig struct { 38 | Logging zap.Config 39 | } 40 | 41 | type zapLogger struct { 42 | *zap.SugaredLogger 43 | } 44 | 45 | func configureZap(level int, production bool) (Logger, error) { 46 | var cfg zapConfig 47 | cfg.Logging = zap.NewProductionConfig() 48 | if !production { 49 | cfg.Logging = zap.NewDevelopmentConfig() 50 | } 51 | 52 | cfg.Logging.Level = zap.NewAtomicLevelAt(zapcore.Level(Debug - level - 1)) 53 | 54 | if err := config.LoadSection(&cfg); err != nil { 55 | return nil, err 56 | } 57 | 58 | l, err := cfg.Logging.Build(zap.AddCallerSkip(2)) 59 | if err != nil { 60 | return nil, err 61 | } 62 | return &zapLogger{l.Sugar()}, nil 63 | 64 | } 65 | 66 | //FIXME: Need to set zap.AddCallerSkip(0) to show correct line numbers for non 67 | //default loggers 68 | func (l *zapLogger) WithFields(keyValues Fields) Logger { 69 | var f = make([]interface{}, 0) 70 | for k, v := range keyValues { 71 | f = append(f, k) 72 | f = append(f, v) 73 | } 74 | n := l.With(f...) 75 | return &zapLogger{n} 76 | } 77 | -------------------------------------------------------------------------------- /log/zap_test.go: -------------------------------------------------------------------------------- 1 | // Copyright (c) 2017 Uber Technologies, Inc. 2 | // 3 | // Permission is hereby granted, free of charge, to any person obtaining a copy 4 | // of this software and associated documentation files (the "Software"), to deal 5 | // in the Software without restriction, including without limitation the rights 6 | // to use, copy, modify, merge, publish, distribute, sublicense, and/or sell 7 | // copies of the Software, and to permit persons to whom the Software is 8 | // furnished to do so, subject to the following conditions: 9 | // 10 | // The above copyright notice and this permission notice shall be included in 11 | // all copies or substantial portions of the Software. 12 | // 13 | // THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR 14 | // IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, 15 | // FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE 16 | // AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER 17 | // LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, 18 | // OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN 19 | // THE SOFTWARE. 20 | 21 | //go:build go1.7 22 | // +build go1.7 23 | 24 | package log 25 | 26 | import ( 27 | "fmt" 28 | "testing" 29 | ) 30 | 31 | func TestZapBasic(t *testing.T) { 32 | for i := 0; i <= Debug; i++ { 33 | fmt.Printf("Level: %v\n", i) 34 | 35 | var err error 36 | def, err = configureZap(i, false) 37 | if err != nil { 38 | t.Fatalf("Init failed %v", err.Error()) 39 | } 40 | 41 | Debugf("msg") 42 | Infof("msg") 43 | Warnf("msg") 44 | Errorf("msg") 45 | 46 | lf := WithFields(Fields{"Test field 1": "val1", "Test field2": "val2"}) 47 | 48 | lf.Debugf("msg with fields") 49 | lf.Infof("msg with fields") 50 | lf.Warnf("msg with fields") 51 | lf.Errorf("msg with fields") 52 | } 53 | } 54 | -------------------------------------------------------------------------------- /main.go: -------------------------------------------------------------------------------- 1 | // Copyright (c) 2017 Uber Technologies, Inc. 2 | // 3 | // Permission is hereby granted, free of charge, to any person obtaining a copy 4 | // of this software and associated documentation files (the "Software"), to deal 5 | // in the Software without restriction, including without limitation the rights 6 | // to use, copy, modify, merge, publish, distribute, sublicense, and/or sell 7 | // copies of the Software, and to permit persons to whom the Software is 8 | // furnished to do so, subject to the following conditions: 9 | // 10 | // The above copyright notice and this permission notice shall be included in 11 | // all copies or substantial portions of the Software. 12 | // 13 | // THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR 14 | // IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, 15 | // FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE 16 | // AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER 17 | // LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, 18 | // OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN 19 | // THE SOFTWARE. 20 | 21 | package main 22 | 23 | import ( 24 | "time" 25 | 26 | "github.com/uber/storagetapper/changelog" 27 | "github.com/uber/storagetapper/config" 28 | "github.com/uber/storagetapper/db" 29 | "github.com/uber/storagetapper/encoder" 30 | "github.com/uber/storagetapper/log" 31 | "github.com/uber/storagetapper/metrics" 32 | "github.com/uber/storagetapper/pipe" 33 | "github.com/uber/storagetapper/pool" 34 | "github.com/uber/storagetapper/server" 35 | "github.com/uber/storagetapper/shutdown" 36 | "github.com/uber/storagetapper/state" 37 | "github.com/uber/storagetapper/streamer" 38 | "github.com/uber/storagetapper/types" 39 | 40 | "golang.org/x/net/context" 41 | ) 42 | 43 | var version, revision = "1.0", "" 44 | 45 | func idle(interval time.Duration) { 46 | metrics.IdleWorkers.Inc() 47 | defer metrics.IdleWorkers.Dec() 48 | 49 | log.Debugf("going idle") 50 | 51 | select { 52 | case <-shutdown.InitiatedCh(): 53 | case <-time.After(interval): 54 | } 55 | } 56 | 57 | func worker(ctx context.Context, cfg *config.AppConfig, inP pipe.Pipe, tpool pool.Thread) { 58 | log.Debugf("Started worker thread, Total: %+v", shutdown.NumProcs()+1) 59 | for !shutdown.Initiated() && !tpool.Terminate() { 60 | if !changelog.Worker(ctx, cfg, inP, tpool) && !streamer.Worker(cfg, inP) { 61 | idle(cfg.WorkerIdleInterval) 62 | } 63 | } 64 | 65 | log.Debugf("Finished worker thread. Threads remaining %v", shutdown.NumProcs()+1) 66 | } 67 | 68 | // mainLow extracted for the tests to be able to run with different configurations 69 | func mainLow(cfg *config.AppConfig) { 70 | env := config.Environment() 71 | 72 | log.Configure(cfg.LogType, cfg.LogLevel, env == config.EnvProduction || env == config.EnvStaging) 73 | 74 | log.Infof("%s Version: %s, Revision: %s", types.MySvcName, version, revision) 75 | 76 | log.Debugf("Config: %+v", cfg) 77 | 78 | types.MyDBName = cfg.StateDBName 79 | types.MyClusterName = cfg.StateClusterName 80 | 81 | err := metrics.Init() 82 | log.F(err) 83 | 84 | shutdown.Setup() 85 | 86 | //This is to resolve package cyclic dependencies 87 | encoder.GetLatestSchema = state.SchemaGet 88 | db.BuiltinResolveCluster = state.ConnectInfoGet 89 | 90 | if err := state.InitManager(shutdown.Context, cfg); err != nil { 91 | log.F(err) 92 | } 93 | 94 | go server.StartHTTPServer(cfg.PortDyn) 95 | 96 | nprocs := uint(cfg.MaxNumProcs) 97 | 98 | if cfg.ChangelogPipeType == "local" { 99 | nprocs = 1 /*Start changelog reader only, it'll control the size of the thread pool*/ 100 | } 101 | 102 | //Increasing batch size is important to prevent Pipes from preserving 103 | //offsets before batch of the size batch_size has been committed to the 104 | //output pipe 105 | //There may be total 2*batch_size+1 events in flight 106 | // * batch_size - in outP batch buffer 107 | // * batch_size - in streamer helper channel buffer 108 | // * +1 - waiting in streamer helper to be pushed when buffer is full 109 | p := cfg.Pipe 110 | p.MaxBatchSize = 2*p.MaxBatchSize + 1 111 | inP, err := pipe.Create(cfg.ChangelogPipeType, &p, state.GetDB()) 112 | log.F(err) 113 | 114 | tp := pool.Create() 115 | 116 | tp.Start(nprocs, func() { 117 | worker(shutdown.Context, cfg, inP, tp) 118 | }) 119 | 120 | shutdown.Wait() 121 | 122 | server.Shutdown() 123 | 124 | pipe.CacheDestroy() 125 | 126 | log.Debugf("FINISHED") 127 | } 128 | 129 | func main() { 130 | cfg := config.Get() 131 | mainLow(cfg) 132 | } 133 | -------------------------------------------------------------------------------- /metrics/counter.go: -------------------------------------------------------------------------------- 1 | // Copyright (c) 2017 Uber Technologies, Inc. 2 | // 3 | // Permission is hereby granted, free of charge, to any person obtaining a copy 4 | // of this software and associated documentation files (the "Software"), to deal 5 | // in the Software without restriction, including without limitation the rights 6 | // to use, copy, modify, merge, publish, distribute, sublicense, and/or sell 7 | // copies of the Software, and to permit persons to whom the Software is 8 | // furnished to do so, subject to the following conditions: 9 | // 10 | // The above copyright notice and this permission notice shall be included in 11 | // all copies or substantial portions of the Software. 12 | // 13 | // THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR 14 | // IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, 15 | // FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE 16 | // AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER 17 | // LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, 18 | // OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN 19 | // THE SOFTWARE. 20 | 21 | package metrics 22 | 23 | import "sync/atomic" 24 | 25 | //Counter is a stateful counter statistics variable 26 | type Counter struct { 27 | backend counter 28 | value int64 29 | } 30 | 31 | //CounterInit is a constructor for Counter 32 | func CounterInit(s scope, name string) *Counter { 33 | var p Counter 34 | p.backend = s.InitCounter(name) 35 | return &p 36 | } 37 | 38 | //Inc increments the value by v 39 | func (p *Counter) Inc(v int64) { 40 | atomic.AddInt64(&p.value, v) 41 | p.backend.Update(atomic.LoadInt64(&p.value)) 42 | } 43 | 44 | //Dec increments the value by v 45 | func (p *Counter) Dec(v int64) { 46 | atomic.AddInt64(&p.value, -v) 47 | p.backend.Update(atomic.LoadInt64(&p.value)) 48 | } 49 | 50 | //Set sets the value of the counter gauge to a specific value 51 | func (p *Counter) Set(v int64) { 52 | atomic.StoreInt64(&p.value, v) 53 | p.backend.Update(atomic.LoadInt64(&p.value)) 54 | } 55 | 56 | //Get returns current value of the counter 57 | func (p *Counter) Get() int64 { 58 | return atomic.LoadInt64(&p.value) 59 | } 60 | 61 | //Emit current value 62 | func (p *Counter) Emit() { 63 | p.backend.Update(atomic.LoadInt64(&p.value)) 64 | } 65 | -------------------------------------------------------------------------------- /metrics/metrics_test.go: -------------------------------------------------------------------------------- 1 | // Copyright (c) 2017 Uber Technologies, Inc. 2 | // 3 | // Permission is hereby granted, free of charge, to any person obtaining a copy 4 | // of this software and associated documentation files (the "Software"), to deal 5 | // in the Software without restriction, including without limitation the rights 6 | // to use, copy, modify, merge, publish, distribute, sublicense, and/or sell 7 | // copies of the Software, and to permit persons to whom the Software is 8 | // furnished to do so, subject to the following conditions: 9 | // 10 | // The above copyright notice and this permission notice shall be included in 11 | // all copies or substantial portions of the Software. 12 | // 13 | // THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR 14 | // IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, 15 | // FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE 16 | // AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER 17 | // LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, 18 | // OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN 19 | // THE SOFTWARE. 20 | 21 | package metrics 22 | 23 | import ( 24 | "os" 25 | "path" 26 | "runtime" 27 | "testing" 28 | "time" 29 | 30 | "github.com/uber/storagetapper/config" 31 | "github.com/uber/storagetapper/log" 32 | ) 33 | 34 | var cfg *config.AppConfig 35 | 36 | //TODO: Write some meaningful tests 37 | 38 | //This is a copy from test/env.go to avoid dependency cycle 39 | func assert(t *testing.T, cond bool) { 40 | if !cond { 41 | pc, file, no, _ := runtime.Caller(1) 42 | details := runtime.FuncForPC(pc) 43 | log.Fatalf("%v:%v %v", path.Base(file), no, path.Base(details.Name())) 44 | t.FailNow() 45 | } 46 | } 47 | 48 | func testMetrics(c metricsConstructor, t *testing.T) { 49 | metricsInit = c 50 | if err := Init(); err != nil { 51 | t.Fatalf("Init failed") 52 | } 53 | 54 | b := NewChangelogReaderMetrics(map[string]string{"process": "ChangelogReader"}) 55 | st := NewStreamerMetrics(map[string]string{"process": "Streamer"}) 56 | sn := NewSnapshotMetrics("", map[string]string{"process": "Snapshot"}) 57 | 58 | b.NumTablesIngesting.Inc(1) 59 | assert(t, int64(1) == b.NumTablesIngesting.Get()) 60 | 61 | sn.NumWorkers.Inc() 62 | assert(t, int64(1) == sn.NumWorkers.Get()) 63 | sn.NumWorkers.Dec() 64 | assert(t, int64(0) == sn.NumWorkers.Get()) 65 | 66 | b.NumWorkers.Inc() 67 | assert(t, int64(1) == b.NumWorkers.Get()) 68 | b.NumWorkers.Dec() 69 | assert(t, int64(0) == b.NumWorkers.Get()) 70 | 71 | st.NumWorkers.Inc() 72 | assert(t, int64(1) == st.NumWorkers.Get()) 73 | st.NumWorkers.Dec() 74 | assert(t, int64(0) == st.NumWorkers.Get()) 75 | 76 | IdleWorkers.Inc() 77 | assert(t, int64(1) == IdleWorkers.Get()) 78 | IdleWorkers.Dec() 79 | assert(t, int64(0) == IdleWorkers.Get()) 80 | 81 | b.EventsRead.Inc(1) 82 | b.ChangelogRowEventsWritten.Inc(1) 83 | b.ChangelogQueryEventsWritten.Inc(1) 84 | b.ChangelogUnhandledEvents.Inc(1) 85 | 86 | st.EventsRead.Inc(1) 87 | st.EventsWritten.Inc(1) 88 | st.BytesRead.Inc(1) 89 | st.BytesWritten.Inc(1) 90 | 91 | sn.EventsRead.Inc(1) 92 | sn.EventsWritten.Inc(1) 93 | sn.BytesRead.Inc(1) 94 | sn.BytesWritten.Inc(10) 95 | 96 | st.TimeInBuffer.Record(1) 97 | 98 | st.TimeInBuffer.Start() 99 | time.Sleep(1 * time.Millisecond) 100 | st.TimeInBuffer.Stop() 101 | } 102 | 103 | func TestMetricsBasic(t *testing.T) { 104 | testMetrics(noopMetricsInit, t) 105 | testMetrics(tallyMetricsInit, t) 106 | } 107 | 108 | func TestMain(m *testing.M) { 109 | cfg = config.Get() 110 | if cfg == nil { 111 | log.Fatalf("Can't load config") 112 | } 113 | 114 | os.Exit(m.Run()) 115 | log.Debugf("Starting shutdown") 116 | } 117 | -------------------------------------------------------------------------------- /metrics/noop.go: -------------------------------------------------------------------------------- 1 | // Copyright (c) 2017 Uber Technologies, Inc. 2 | // 3 | // Permission is hereby granted, free of charge, to any person obtaining a copy 4 | // of this software and associated documentation files (the "Software"), to deal 5 | // in the Software without restriction, including without limitation the rights 6 | // to use, copy, modify, merge, publish, distribute, sublicense, and/or sell 7 | // copies of the Software, and to permit persons to whom the Software is 8 | // furnished to do so, subject to the following conditions: 9 | // 10 | // The above copyright notice and this permission notice shall be included in 11 | // all copies or substantial portions of the Software. 12 | // 13 | // THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR 14 | // IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, 15 | // FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE 16 | // AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER 17 | // LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, 18 | // OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN 19 | // THE SOFTWARE. 20 | 21 | package metrics 22 | 23 | import "time" 24 | 25 | type noopMetrics struct { 26 | } 27 | 28 | func (n *noopMetrics) InitTimer(name string) timer { 29 | var t noopTimer 30 | return &t 31 | } 32 | 33 | func (n *noopMetrics) InitCounter(name string) counter { 34 | return &noopCounter{} 35 | } 36 | 37 | func (n *noopMetrics) SubScope(name string) scope { 38 | return n 39 | } 40 | 41 | func (n *noopMetrics) Tagged(tags map[string]string) scope { 42 | return n 43 | } 44 | 45 | func noopMetricsInit() (scope, error) { 46 | var n noopMetrics 47 | return &n, nil 48 | } 49 | 50 | type noopCounter struct { 51 | } 52 | 53 | func (u *noopCounter) Update(value int64) { 54 | } 55 | 56 | func (u *noopCounter) Tag(tags map[string]string) { 57 | } 58 | 59 | func (u *noopCounter) Tagged(tags map[string]string) counter { 60 | return &noopCounter{} 61 | } 62 | 63 | type noopTimer struct { 64 | } 65 | 66 | func (u *noopTimer) Start() { 67 | } 68 | 69 | func (u *noopTimer) Stop() { 70 | } 71 | 72 | func (u *noopTimer) Record(value time.Duration) { 73 | } 74 | 75 | func (u *noopTimer) Tag(tags map[string]string) { 76 | } 77 | -------------------------------------------------------------------------------- /metrics/process.go: -------------------------------------------------------------------------------- 1 | // Copyright (c) 2017 Uber Technologies, Inc. 2 | // 3 | // Permission is hereby granted, free of charge, to any person obtaining a copy 4 | // of this software and associated documentation files (the "Software"), to deal 5 | // in the Software without restriction, including without limitation the rights 6 | // to use, copy, modify, merge, publish, distribute, sublicense, and/or sell 7 | // copies of the Software, and to permit persons to whom the Software is 8 | // furnished to do so, subject to the following conditions: 9 | // 10 | // The above copyright notice and this permission notice shall be included in 11 | // all copies or substantial portions of the Software. 12 | // 13 | // THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR 14 | // IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, 15 | // FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE 16 | // AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER 17 | // LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, 18 | // OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN 19 | // THE SOFTWARE. 20 | 21 | package metrics 22 | 23 | import "sync/atomic" 24 | 25 | //ProcessCounter counts number of started, finished and currently running 26 | //processes 27 | type ProcessCounter struct { 28 | MStarted *Counter 29 | MFinished *Counter 30 | MRunning *Counter 31 | 32 | Started int64 33 | Finished int64 34 | } 35 | 36 | //ProcessCounterInit is the constructor for ProcessCounter 37 | func ProcessCounterInit(c scope, name string) *ProcessCounter { 38 | var p ProcessCounter 39 | p.MStarted = CounterInit(c, name+"_started") 40 | p.MFinished = CounterInit(c, name+"_finished") 41 | p.MRunning = CounterInit(c, name+"_running") 42 | return &p 43 | } 44 | 45 | //Inc increments number of processes started and reports metric for number of 46 | //started and running processes 47 | func (p *ProcessCounter) Inc() { 48 | atomic.AddInt64(&p.Started, 1) 49 | p.MStarted.Set(atomic.LoadInt64(&p.Started)) 50 | p.MRunning.Set(atomic.LoadInt64(&p.Started) - atomic.LoadInt64(&p.Finished)) 51 | } 52 | 53 | //Dec decrements number of processes finished and reports metric for number of 54 | //finished and running processes 55 | func (p *ProcessCounter) Dec() { 56 | atomic.AddInt64(&p.Finished, 1) 57 | p.MFinished.Set(atomic.LoadInt64(&p.Finished)) 58 | p.MRunning.Set(atomic.LoadInt64(&p.Started) - atomic.LoadInt64(&p.Finished)) 59 | } 60 | 61 | //Emit current value 62 | func (p *ProcessCounter) Emit() { 63 | p.MStarted.Set(atomic.LoadInt64(&p.Started)) 64 | p.MFinished.Set(atomic.LoadInt64(&p.Finished)) 65 | p.MRunning.Set(atomic.LoadInt64(&p.Started) - atomic.LoadInt64(&p.Finished)) 66 | } 67 | 68 | //Get returns number of currently running processes 69 | func (p *ProcessCounter) Get() int64 { 70 | return atomic.LoadInt64(&p.Started) - atomic.LoadInt64(&p.Finished) 71 | } 72 | -------------------------------------------------------------------------------- /metrics/tally.go: -------------------------------------------------------------------------------- 1 | // Copyright (c) 2017 Uber Technologies, Inc. 2 | // 3 | // Permission is hereby granted, free of charge, to any person obtaining a copy 4 | // of this software and associated documentation files (the "Software"), to deal 5 | // in the Software without restriction, including without limitation the rights 6 | // to use, copy, modify, merge, publish, distribute, sublicense, and/or sell 7 | // copies of the Software, and to permit persons to whom the Software is 8 | // furnished to do so, subject to the following conditions: 9 | // 10 | // The above copyright notice and this permission notice shall be included in 11 | // all copies or substantial portions of the Software. 12 | // 13 | // THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR 14 | // IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, 15 | // FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE 16 | // AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER 17 | // LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, 18 | // OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN 19 | // THE SOFTWARE. 20 | 21 | package metrics 22 | 23 | import ( 24 | "io" 25 | "time" 26 | 27 | "github.com/cactus/go-statsd-client/statsd" 28 | "github.com/uber-go/tally" 29 | tallystatsd "github.com/uber-go/tally/statsd" 30 | 31 | "github.com/uber/storagetapper/config" 32 | "github.com/uber/storagetapper/log" 33 | "github.com/uber/storagetapper/types" 34 | ) 35 | 36 | //TallyConfiguration ... 37 | type TallyConfiguration struct { 38 | Address string 39 | Interval time.Duration 40 | } 41 | 42 | type tallyMetricsConfig struct { 43 | Metrics struct { 44 | Tally TallyConfiguration 45 | } 46 | } 47 | 48 | type tallyMetrics struct { 49 | RootScope tally.Scope 50 | } 51 | 52 | func (t *tallyMetrics) InitTimer(name string) timer { 53 | var tt tallyTimer 54 | tt.timer = t.RootScope.Timer(name) 55 | return &tt 56 | } 57 | 58 | func (t *tallyMetrics) InitCounter(name string) counter { 59 | var c tallyCounter 60 | c.counter = t.RootScope.Gauge(name) 61 | return &c 62 | } 63 | 64 | func (t *tallyMetrics) SubScope(name string) scope { 65 | return &tallyMetrics{t.RootScope.SubScope(name)} 66 | } 67 | 68 | func (t *tallyMetrics) Tagged(tags map[string]string) scope { 69 | return &tallyMetrics{t.RootScope.Tagged(tags)} 70 | } 71 | 72 | func newScope(address string, interval time.Duration) (tally.Scope, io.Closer) { 73 | config := &statsd.ClientConfig{Address: address, Prefix: "stats", FlushInterval: -1, FlushBytes: 0} 74 | 75 | statter, _ := statsd.NewClientWithConfig(config) 76 | 77 | scope, closer := tally.NewRootScope(tally.ScopeOptions{ 78 | Prefix: types.MySvcName, 79 | Tags: map[string]string{}, 80 | Reporter: tallystatsd.NewReporter(statter, tallystatsd.Options{}), 81 | }, interval) 82 | 83 | return scope, closer 84 | } 85 | 86 | func tallyMetricsInit() (scope, error) { 87 | var cfg tallyMetricsConfig 88 | 89 | if err := config.LoadSection(&cfg); err != nil { 90 | return nil, err 91 | } 92 | 93 | log.Debugf("Metrics config: %+v", cfg) 94 | 95 | s, _ := newScope(cfg.Metrics.Tally.Address, cfg.Metrics.Tally.Interval) 96 | 97 | m := &tallyMetrics{ 98 | RootScope: s, 99 | } 100 | 101 | return m, nil 102 | } 103 | 104 | type tallyCounter struct { 105 | counter tally.Gauge 106 | } 107 | 108 | func (t *tallyCounter) Update(value int64) { 109 | t.counter.Update(float64(value)) 110 | } 111 | 112 | type tallyTimer struct { 113 | timer tally.Timer 114 | w tally.Stopwatch 115 | } 116 | 117 | func (t *tallyTimer) Start() { 118 | t.w = t.timer.Start() 119 | } 120 | 121 | func (t *tallyTimer) Stop() { 122 | t.w.Stop() 123 | } 124 | 125 | func (t *tallyTimer) Record(value time.Duration) { 126 | t.timer.Record(value) 127 | } 128 | -------------------------------------------------------------------------------- /metrics/timer.go: -------------------------------------------------------------------------------- 1 | // Copyright (c) 2017 Uber Technologies, Inc. 2 | // 3 | // Permission is hereby granted, free of charge, to any person obtaining a copy 4 | // of this software and associated documentation files (the "Software"), to deal 5 | // in the Software without restriction, including without limitation the rights 6 | // to use, copy, modify, merge, publish, distribute, sublicense, and/or sell 7 | // copies of the Software, and to permit persons to whom the Software is 8 | // furnished to do so, subject to the following conditions: 9 | // 10 | // The above copyright notice and this permission notice shall be included in 11 | // all copies or substantial portions of the Software. 12 | // 13 | // THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR 14 | // IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, 15 | // FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE 16 | // AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER 17 | // LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, 18 | // OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN 19 | // THE SOFTWARE. 20 | 21 | package metrics 22 | 23 | import "time" 24 | 25 | //Timer is a wrapper around metrics.Timer 26 | type Timer struct { 27 | backend timer 28 | } 29 | 30 | //TimerInit is a constructor for Timer 31 | func TimerInit(s scope, name string) *Timer { 32 | var t Timer 33 | t.backend = s.InitTimer(name) 34 | return &t 35 | } 36 | 37 | //Start starts the timer 38 | func (t *Timer) Start() *Timer { 39 | t.backend.Start() 40 | return t 41 | } 42 | 43 | //Stop stops the timer 44 | func (t *Timer) Stop() { 45 | t.backend.Stop() 46 | } 47 | 48 | //Record sets the value of the timer to a specific value 49 | func (t *Timer) Record(v time.Duration) { 50 | t.backend.Record(v) 51 | } 52 | -------------------------------------------------------------------------------- /pipe/cache.go: -------------------------------------------------------------------------------- 1 | package pipe 2 | 3 | import ( 4 | "crypto/sha256" 5 | "database/sql" 6 | "encoding/json" 7 | "fmt" 8 | "reflect" 9 | "sync" 10 | 11 | "github.com/uber/storagetapper/config" 12 | "github.com/uber/storagetapper/log" 13 | ) 14 | 15 | type cacheEntry struct { 16 | pipe Pipe 17 | cfg config.PipeConfig 18 | } 19 | 20 | var cache map[string]cacheEntry 21 | var lock sync.Mutex 22 | 23 | // CacheGet returns an instance of pipe with specified config from cache or 24 | // creates new one if it's not in the cache yet 25 | func CacheGet(pipeType string, cfg *config.PipeConfig, db *sql.DB) (Pipe, error) { 26 | lock.Lock() 27 | defer lock.Unlock() 28 | 29 | if cache == nil { 30 | cache = make(map[string]cacheEntry) 31 | } 32 | 33 | b, err := json.Marshal(cfg) 34 | if err != nil { 35 | return nil, err 36 | } 37 | 38 | h := sha256.New() 39 | _, _ = h.Write([]byte(pipeType + "$$$" + fmt.Sprintf("%p", db) + "$$$")) 40 | _, _ = h.Write(b) 41 | hs := fmt.Sprintf("%0x", h.Sum(nil)) 42 | 43 | p, ok := cache[hs] 44 | if ok && reflect.DeepEqual(cfg, &p.cfg) { 45 | return p.pipe, nil 46 | } 47 | 48 | //FIXME: Implement proper collisions handling 49 | 50 | np, err := Create(pipeType, cfg, db) 51 | if err != nil { 52 | return nil, err 53 | } 54 | 55 | cache[hs] = cacheEntry{np, *cfg} 56 | 57 | log.Debugf("Created and cached new '%v' pipe (hash %v) with config: %+v. Cache size %v", pipeType, hs, *cfg, len(cache)) 58 | 59 | return np, nil 60 | } 61 | 62 | // CacheDestroy releases all resources associated with cached pipes 63 | func CacheDestroy() { 64 | lock.Lock() 65 | defer lock.Unlock() 66 | 67 | for h, p := range cache { 68 | log.Debugf("Closing %v pipe (hash %v) with config %+v", p.pipe.Type(), h, p.cfg) 69 | log.E(p.pipe.Close()) 70 | } 71 | 72 | cache = nil 73 | } 74 | -------------------------------------------------------------------------------- /pipe/cache_test.go: -------------------------------------------------------------------------------- 1 | package pipe 2 | 3 | import ( 4 | "database/sql" 5 | "testing" 6 | 7 | "github.com/uber/storagetapper/config" 8 | "github.com/uber/storagetapper/log" 9 | "github.com/uber/storagetapper/test" 10 | ) 11 | 12 | var count int 13 | 14 | func testPipeInit(cfg *config.PipeConfig, db *sql.DB) (Pipe, error) { 15 | log.Debugf("testPipeInit %v", count) 16 | count++ 17 | c := *cfg 18 | c.EndOfStreamMark = true 19 | return &localPipe{cfg: c}, nil 20 | } 21 | 22 | func TestPipeCacheBasic(t *testing.T) { 23 | pcfg := cfg.Pipe 24 | p, err := CacheGet("local", &pcfg, nil) 25 | test.CheckFail(err, t) 26 | test.Assert(t, p.Type() == "local", "should be local type") 27 | _, err = CacheGet("local", &pcfg, nil) 28 | test.CheckFail(err, t) 29 | test.Assert(t, len(cache) == 1, "second local CacheGet shouldn't create an entry") 30 | 31 | p, err = CacheGet("file", &pcfg, nil) 32 | test.CheckFail(err, t) 33 | test.Assert(t, len(cache) == 2, "an entry should be created for first file pipe") 34 | test.Assert(t, p.Type() == "file", "should be file type") 35 | _, err = CacheGet("file", &pcfg, nil) 36 | test.CheckFail(err, t) 37 | test.Assert(t, len(cache) == 2, "second file CacheGet shouldn't create an entry") 38 | 39 | pcfg1 := cfg.Pipe 40 | pcfg1.BaseDir = "/tmp/somedir" 41 | 42 | p, err = CacheGet("file", &pcfg1, nil) 43 | test.CheckFail(err, t) 44 | test.Assert(t, p.Type() == "file", "should be file type") 45 | test.Assert(t, len(cache) == 3, "different config, should create an entry") 46 | test.Assert(t, p.Config().BaseDir == "/tmp/somedir", "check that config is ours") 47 | 48 | p, err = CacheGet("file", &pcfg, nil) 49 | test.CheckFail(err, t) 50 | test.Assert(t, p.Type() == "file", "should be file type") 51 | test.Assert(t, p.Config().BaseDir != "/tmp/somedir", "check that we can still get pipe with original config") 52 | 53 | Pipes["testpipe"] = testPipeInit 54 | for i := 0; i < 5; i++ { 55 | _, _ = CacheGet("testpipe", &pcfg, nil) 56 | } 57 | log.Debugf("%v", count) 58 | test.Assert(t, count == 1, "testPipe should be created just once") 59 | 60 | CacheDestroy() 61 | 62 | test.Assert(t, len(cache) == 0, "destroyed") 63 | } 64 | -------------------------------------------------------------------------------- /pipe/hdfs_test.go: -------------------------------------------------------------------------------- 1 | package pipe 2 | 3 | import ( 4 | "os" 5 | "testing" 6 | 7 | "github.com/efirs/hdfs/v2" 8 | "github.com/uber/storagetapper/test" 9 | ) 10 | 11 | func deleteTestHdfsTopics(t *testing.T) { 12 | cp := hdfs.ClientOptions{User: cfg.Pipe.Hadoop.User, Addresses: cfg.Pipe.Hadoop.Addresses} 13 | client, err := hdfs.NewClient(cp) 14 | test.CheckFail(err, t) 15 | 16 | err = client.RemoveAll(cfg.Pipe.Hadoop.BaseDir) 17 | if !os.IsNotExist(err) { 18 | test.CheckFail(err, t) 19 | } 20 | 21 | err = client.MkdirAll(cfg.Pipe.Hadoop.BaseDir, 0770) //Pipe calls mkdirall so this may not be needed 22 | test.CheckFail(err, t) 23 | } 24 | 25 | func testHdfsBasic(size int64, pubKey string, privKey string, signKey string, t *testing.T) { 26 | pcfg := cfg.Pipe 27 | pcfg.FileDelimited = true 28 | p, err := initHdfsPipe(&pcfg, nil) 29 | test.CheckFail(err, t) 30 | h, ok := p.(*hdfsPipe) 31 | test.Assert(t, ok, "Unexpected pipe type") 32 | h.cfg.Encryption.PrivateKey = privKey 33 | h.cfg.Encryption.PublicKey = pubKey 34 | h.cfg.Encryption.SigningKey = privKey 35 | 36 | startCh = make(chan bool) 37 | 38 | deleteTestHdfsTopics(t) 39 | testLoop(p, t, NOKEY) 40 | 41 | deleteTestHdfsTopics(t) 42 | testLoop(p, t, KEY) 43 | } 44 | 45 | func TestHdfsBasic(t *testing.T) { 46 | testHdfsBasic(1024, "", "", "", t) 47 | } 48 | 49 | func TestHdfsSmall(t *testing.T) { 50 | testHdfsBasic(1, "", "", "", t) 51 | } 52 | 53 | func TestHdfsType(t *testing.T) { 54 | pt := "hdfs" 55 | p, err := initHdfsPipe(&cfg.Pipe, nil) 56 | test.CheckFail(err, t) 57 | test.Assert(t, p.Type() == pt, "type should be "+pt) 58 | } 59 | 60 | func TestHdfsEncryption(t *testing.T) { 61 | pubKey, privKey := genTestKeys(t) 62 | testHdfsBasic(1, pubKey, privKey, privKey, t) 63 | } 64 | 65 | func TestHdfsCompression(t *testing.T) { 66 | cfg.Pipe.Compression = true 67 | defer func() { cfg.Pipe.Compression = false }() 68 | testHdfsBasic(1, "", "", "", t) 69 | } 70 | 71 | func TestHdfsCompressionAndEncryption(t *testing.T) { 72 | cfg.Pipe.Compression = true 73 | defer func() { cfg.Pipe.Compression = false }() 74 | pubKey, privKey := genTestKeys(t) 75 | testHdfsBasic(1, pubKey, privKey, privKey, t) 76 | } 77 | -------------------------------------------------------------------------------- /pipe/header.go: -------------------------------------------------------------------------------- 1 | package pipe 2 | 3 | //Header represent file metadata in the beginning of the file 4 | type Header struct { 5 | Format string 6 | Filters []string `json:",omitempty"` 7 | Schema []byte `json:",omitempty"` 8 | Delimited bool `json:",omitempty"` 9 | HMAC string `json:"HMAC-SHA256,omitempty"` 10 | IV string `json:"AES256-CFB-IV,omitempty"` 11 | } 12 | 13 | /* 14 | func writeHeader(header *Header, hash []byte, f io.Writer) error { 15 | if len(hash) != 0 { 16 | header.HMAC = fmt.Sprintf("%x", hash) 17 | } 18 | 19 | h, err := json.Marshal(header) 20 | if err != nil { 21 | return err 22 | } 23 | 24 | h = append(h, delimiter) 25 | 26 | _, err = f.Write(h) 27 | 28 | return err 29 | } 30 | 31 | func readHeader(r *bufio.Reader) (Header, error) { 32 | u := &Header{} 33 | 34 | h, err := r.ReadBytes(delimiter) 35 | if err != nil { 36 | return *u, err 37 | } 38 | 39 | err = json.Unmarshal(h, u) 40 | if err != nil { 41 | return *u, err 42 | } 43 | 44 | return *u, nil 45 | } 46 | */ 47 | -------------------------------------------------------------------------------- /pipe/local_test.go: -------------------------------------------------------------------------------- 1 | // Copyright (c) 2017 Uber Technologies, Inc. 2 | // 3 | // Permission is hereby granted, free of charge, to any person obtaining a copy 4 | // of this software and associated documentation files (the "Software"), to deal 5 | // in the Software without restriction, including without limitation the rights 6 | // to use, copy, modify, merge, publish, distribute, sublicense, and/or sell 7 | // copies of the Software, and to permit persons to whom the Software is 8 | // furnished to do so, subject to the following conditions: 9 | // 10 | // The above copyright notice and this permission notice shall be included in 11 | // all copies or substantial portions of the Software. 12 | // 13 | // THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR 14 | // IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, 15 | // FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE 16 | // AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER 17 | // LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, 18 | // OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN 19 | // THE SOFTWARE. 20 | 21 | package pipe 22 | 23 | import ( 24 | "bytes" 25 | "os" 26 | "strconv" 27 | "sync/atomic" 28 | "testing" 29 | 30 | "github.com/uber/storagetapper/config" 31 | "github.com/uber/storagetapper/log" 32 | "github.com/uber/storagetapper/shutdown" 33 | "github.com/uber/storagetapper/state" 34 | "github.com/uber/storagetapper/test" 35 | ) 36 | 37 | var cfg *config.AppConfig 38 | 39 | func localConsumer(p Pipe, key string, cerr *int64) { 40 | defer wg.Done() 41 | c, err := p.NewConsumer(key) 42 | if log.E(err) { 43 | atomic.AddInt64(cerr, 1) 44 | return 45 | } 46 | var i int 47 | for { 48 | in, err := c.FetchNext() 49 | if log.E(err) { 50 | atomic.AddInt64(cerr, 1) 51 | return 52 | } 53 | if in == nil { 54 | break 55 | } 56 | b := in.([]byte) 57 | if len(b) == 0 { 58 | break 59 | } 60 | n := bytes.IndexByte(b, 0) 61 | if n == -1 { 62 | n = len(b) 63 | } 64 | s := string(b[:n]) 65 | if s != key+"."+strconv.Itoa(i) { 66 | log.Debugf("Received: %v", s) 67 | log.Debugf("Expected: %v", key+"."+strconv.Itoa(i)) 68 | atomic.AddInt64(cerr, 1) 69 | return 70 | } 71 | i++ 72 | } 73 | } 74 | 75 | func localProducer(p Pipe, key string, cerr *int64) { 76 | n := 1000 77 | defer wg.Done() 78 | c, err := p.NewProducer(key) 79 | if log.E(err) { 80 | atomic.AddInt64(cerr, 1) 81 | return 82 | } 83 | for i := 0; i < n; i++ { 84 | msg := key + "." + strconv.Itoa(i) 85 | b := []byte(msg) 86 | err = c.Push(b) 87 | if log.E(err) { 88 | atomic.AddInt64(cerr, 1) 89 | return 90 | } 91 | } 92 | err = c.Push(nil) 93 | if log.E(err) { 94 | atomic.AddInt64(cerr, 1) 95 | return 96 | } 97 | } 98 | 99 | func TestLocalBasic(t *testing.T) { 100 | n := 16 101 | p, err := Create("local", &cfg.Pipe, nil) 102 | test.CheckFail(err, t) 103 | 104 | var cerr int64 105 | 106 | wg.Add(n) 107 | for i := 0; i < n; i++ { 108 | go localConsumer(p, "key"+strconv.Itoa(i), &cerr) 109 | } 110 | 111 | wg.Add(n) 112 | for i := 0; i < n; i++ { 113 | go localProducer(p, "key"+strconv.Itoa(i), &cerr) 114 | } 115 | 116 | wg.Wait() 117 | 118 | if atomic.LoadInt64(&cerr) != 0 { 119 | t.FailNow() 120 | } 121 | } 122 | 123 | func TestLocalType(t *testing.T) { 124 | pt := "local" 125 | p, _ := initLocalPipe(&cfg.Pipe, nil) 126 | test.Assert(t, p.Type() == pt, "type should be "+pt) 127 | } 128 | 129 | func TestLocalPartitionKey(t *testing.T) { 130 | lp, _ := initLocalPipe(&cfg.Pipe, nil) 131 | p, err := lp.NewProducer("partition-key-test-topic") 132 | test.CheckFail(err, t) 133 | key := "some key" 134 | test.Assert(t, p.PartitionKey("log", key) == key, "local pipe should reteurn unmodified key") 135 | key = "other key" 136 | test.Assert(t, p.PartitionKey("snapshot", key) == key, "local pipe should return unmodified key") 137 | } 138 | 139 | func TestMain(m *testing.M) { 140 | cfg = test.LoadConfig() 141 | 142 | shutdown.Setup() 143 | defer func() { 144 | shutdown.Initiate() 145 | shutdown.Wait() 146 | }() 147 | 148 | if err := state.InitManager(shutdown.Context, cfg); err != nil { 149 | log.Fatalf("Failed to init State") 150 | os.Exit(1) 151 | } 152 | defer state.Close() 153 | 154 | os.Exit(m.Run()) 155 | } 156 | -------------------------------------------------------------------------------- /pipe/pipe.go: -------------------------------------------------------------------------------- 1 | // Copyright (c) 2017 Uber Technologies, Inc. 2 | // 3 | // Permission is hereby granted, free of charge, to any person obtaining a copy 4 | // of this software and associated documentation files (the "Software"), to deal 5 | // in the Software without restriction, including without limitation the rights 6 | // to use, copy, modify, merge, publish, distribute, sublicense, and/or sell 7 | // copies of the Software, and to permit persons to whom the Software is 8 | // furnished to do so, subject to the following conditions: 9 | // 10 | // The above copyright notice and this permission notice shall be included in 11 | // all copies or substantial portions of the Software. 12 | // 13 | // THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR 14 | // IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, 15 | // FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE 16 | // AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER 17 | // LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, 18 | // OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN 19 | // THE SOFTWARE. 20 | 21 | package pipe 22 | 23 | import ( 24 | "context" 25 | "database/sql" 26 | "fmt" 27 | "strings" 28 | "sync" 29 | 30 | //"context" 31 | 32 | "github.com/uber/storagetapper/config" 33 | ) 34 | 35 | //Consumer consumer interface for the pipe 36 | type Consumer interface { 37 | Close() error 38 | //CloseOnFailure doesn't save offsets 39 | CloseOnFailure() error 40 | Message() chan interface{} 41 | Error() chan error 42 | FetchNext() (interface{}, error) 43 | //Allows to explicitly persists current consumer position 44 | SaveOffset() error 45 | 46 | //SetFormat allow to tell consumer the format of the file when there is no 47 | //header 48 | SetFormat(format string) 49 | } 50 | 51 | //Producer producer interface for pipe 52 | type Producer interface { 53 | Push(data interface{}) error 54 | PushK(key string, data interface{}) error 55 | PushSchema(key string, data []byte) error 56 | //PushBatch queues the messages instead of sending immediately 57 | PushBatch(key string, data interface{}) error 58 | //PushCommit writes out all the messages queued by PushBatch 59 | PushBatchCommit() error 60 | Close() error 61 | CloseOnFailure() error 62 | 63 | SetFormat(format string) 64 | 65 | PartitionKey(source string, key string) string 66 | } 67 | 68 | //Pipe connects named producers and consumers 69 | type Pipe interface { 70 | NewConsumer(topic string) (Consumer, error) 71 | NewProducer(topic string) (Producer, error) 72 | Type() string 73 | Config() *config.PipeConfig 74 | Close() error 75 | } 76 | 77 | type constructor func(cfg *config.PipeConfig, db *sql.DB) (Pipe, error) 78 | 79 | //Pipes is the list of registered pipes 80 | //Plugins insert their constructors into this map 81 | var Pipes map[string]constructor 82 | 83 | //registerPlugin should be called from plugin's init 84 | func registerPlugin(name string, init constructor) { 85 | if Pipes == nil { 86 | Pipes = make(map[string]constructor) 87 | } 88 | Pipes[name] = init 89 | } 90 | 91 | //Create is a pipe factory 92 | //pctx is used to be able to cancel blocking calls inside pipe, like during 93 | //shutdown 94 | func Create(pipeType string, cfg *config.PipeConfig, db *sql.DB) (Pipe, error) { 95 | 96 | init := Pipes[strings.ToLower(pipeType)] 97 | if init == nil { 98 | return nil, fmt.Errorf("unsupported pipe: %s", strings.ToLower(pipeType)) 99 | } 100 | 101 | pipe, err := init(cfg, db) 102 | if err != nil { 103 | return nil, err 104 | } 105 | 106 | return pipe, nil 107 | } 108 | 109 | type baseConsumer struct { 110 | ctx context.Context 111 | cancel context.CancelFunc 112 | wg sync.WaitGroup 113 | msgCh chan interface{} 114 | errCh chan error 115 | } 116 | 117 | type fetchFunc func() (interface{}, error) 118 | 119 | func (p *baseConsumer) initBaseConsumer(fn fetchFunc) { 120 | p.ctx, p.cancel = context.WithCancel(context.Background()) 121 | p.msgCh = make(chan interface{}) 122 | p.errCh = make(chan error) 123 | 124 | p.wg.Add(1) 125 | go p.fetchLoop(fn) 126 | } 127 | 128 | func (p *baseConsumer) Message() chan interface{} { 129 | return p.msgCh 130 | } 131 | func (p *baseConsumer) Error() chan error { 132 | return p.errCh 133 | } 134 | 135 | func (p *baseConsumer) FetchNext() (interface{}, error) { 136 | select { 137 | case msg := <-p.msgCh: 138 | return msg, nil 139 | case err := <-p.errCh: 140 | return nil, err 141 | case <-p.ctx.Done(): 142 | } 143 | return nil, nil 144 | } 145 | 146 | func (p *baseConsumer) fetchLoop(fn fetchFunc) { 147 | defer p.wg.Done() 148 | for { 149 | msg, err := fn() 150 | if err != nil { 151 | p.sendErr(err) 152 | return 153 | } 154 | if !p.sendMsg(msg) || msg == nil { 155 | return 156 | } 157 | } 158 | } 159 | 160 | func (p *baseConsumer) sendMsg(msg interface{}) bool { 161 | select { 162 | case p.msgCh <- msg: 163 | return true 164 | case <-p.ctx.Done(): 165 | } 166 | return false 167 | } 168 | 169 | func (p *baseConsumer) sendErr(err error) { 170 | select { 171 | case p.errCh <- err: 172 | case <-p.ctx.Done(): 173 | } 174 | } 175 | -------------------------------------------------------------------------------- /pipe/s3_test.go: -------------------------------------------------------------------------------- 1 | package pipe 2 | 3 | import ( 4 | "testing" 5 | 6 | "github.com/gofrs/uuid" 7 | "github.com/uber/storagetapper/log" 8 | "github.com/uber/storagetapper/test" 9 | ) 10 | 11 | func prepareTestS3Topics(t *testing.T) { 12 | u, err := uuid.NewV4() 13 | test.CheckFail(err, t) 14 | cfg.Pipe.S3.BaseDir = "/tmp/s3_pipe_test/" + u.String() 15 | log.Debugf("Test directory %v", cfg.Pipe.S3.BaseDir) 16 | } 17 | 18 | func testS3Basic(size int64, pt int, t *testing.T) { 19 | prepareTestS3Topics(t) 20 | 21 | pcfg := cfg.Pipe 22 | pcfg.FileDelimited = true 23 | p, err := initS3Pipe(&pcfg, nil) 24 | test.CheckFail(err, t) 25 | 26 | startCh = make(chan bool) 27 | 28 | testLoop(p, t, pt) 29 | } 30 | 31 | func TestS3Basic(t *testing.T) { 32 | testS3Basic(1024, NOKEY, t) 33 | testS3Basic(1024, KEY, t) 34 | } 35 | 36 | func TestS3Small(t *testing.T) { 37 | testS3Basic(1, NOKEY, t) 38 | testS3Basic(1, KEY, t) 39 | } 40 | 41 | func TestS3Type(t *testing.T) { 42 | pt := "s3" 43 | p, _ := initS3Pipe(&cfg.Pipe, nil) 44 | test.Assert(t, p.Type() == pt, "type should be "+pt) 45 | } 46 | -------------------------------------------------------------------------------- /pool/pool.go: -------------------------------------------------------------------------------- 1 | // Copyright (c) 2017 Uber Technologies, Inc. 2 | // 3 | // Permission is hereby granted, free of charge, to any person obtaining a copy 4 | // of this software and associated documentation files (the "Software"), to deal 5 | // in the Software without restriction, including without limitation the rights 6 | // to use, copy, modify, merge, publish, distribute, sublicense, and/or sell 7 | // copies of the Software, and to permit persons to whom the Software is 8 | // furnished to do so, subject to the following conditions: 9 | // 10 | // The above copyright notice and this permission notice shall be included in 11 | // all copies or substantial portions of the Software. 12 | // 13 | // THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR 14 | // IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, 15 | // FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE 16 | // AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER 17 | // LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, 18 | // OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN 19 | // THE SOFTWARE. 20 | 21 | package pool 22 | 23 | import ( 24 | "sync" 25 | 26 | "github.com/uber/storagetapper/log" 27 | "github.com/uber/storagetapper/shutdown" 28 | ) 29 | 30 | /*Thread is an implementation of pool interface */ 31 | type Thread interface { 32 | Start(m uint, f func()) 33 | Adjust(m uint) 34 | Terminate() bool 35 | NumProcs() uint 36 | } 37 | 38 | type poolImpl struct { 39 | mutex sync.Mutex 40 | numProcs uint 41 | maxNumProcs uint 42 | fn func() 43 | } 44 | 45 | /*Create helps to hide poolImpl in the package, but not really required */ 46 | func Create() Thread { 47 | return &poolImpl{} 48 | } 49 | 50 | /*Start instantiates a pool of size of 'm' of 'f' goroutines */ 51 | /*Start and Create separation allows to pass pool instance to 'f' goroutine */ 52 | func (p *poolImpl) Start(m uint, f func()) { 53 | p.fn = f 54 | p.Adjust(m) 55 | } 56 | 57 | /*Adjust resizes the pool. It creates new threads if requested size is bigger 58 | * then current size, while it assumes threads cooperation when requested size is 59 | * smaller then current size. Threads should periodically call Terminate function 60 | * and obey the result. */ 61 | func (p *poolImpl) Adjust(m uint) { 62 | p.mutex.Lock() 63 | defer p.mutex.Unlock() 64 | log.Debugf("Current size=%v, current maximum size=%v, requested size=%v", p.numProcs, p.maxNumProcs, m) 65 | p.maxNumProcs = m 66 | if p.numProcs < p.maxNumProcs { 67 | adj := p.maxNumProcs - p.numProcs 68 | shutdown.Register(int32(adj)) 69 | for i := uint(0); i < adj; i++ { 70 | go func() { defer shutdown.Done(); p.fn() }() 71 | } 72 | p.numProcs = m 73 | } 74 | } 75 | 76 | /*Terminate return true if the caller thread need to terminate */ 77 | func (p *poolImpl) Terminate() bool { 78 | //Uncomment if Terminate is called frequently 79 | //Introduces a race when thread can miss Pool resize event, that's ok, so as 80 | //some other threads may see the event, or we will see it on the next 81 | //iteration 82 | // if p.numProcs <= p.maxNumProcs { 83 | // return false 84 | // } 85 | 86 | p.mutex.Lock() 87 | defer p.mutex.Unlock() 88 | 89 | if p.numProcs > p.maxNumProcs { 90 | p.numProcs-- 91 | log.Debugf("Terminating. Current size=%v, current maximum size=%v", p.numProcs, p.maxNumProcs) 92 | return true 93 | } 94 | 95 | return false 96 | } 97 | 98 | /*NumProcs return current size of the pool */ 99 | func (p *poolImpl) NumProcs() uint { 100 | p.mutex.Lock() 101 | defer p.mutex.Unlock() 102 | return p.numProcs 103 | } 104 | -------------------------------------------------------------------------------- /pool/pool_test.go: -------------------------------------------------------------------------------- 1 | // Copyright (c) 2017 Uber Technologies, Inc. 2 | // 3 | // Permission is hereby granted, free of charge, to any person obtaining a copy 4 | // of this software and associated documentation files (the "Software"), to deal 5 | // in the Software without restriction, including without limitation the rights 6 | // to use, copy, modify, merge, publish, distribute, sublicense, and/or sell 7 | // copies of the Software, and to permit persons to whom the Software is 8 | // furnished to do so, subject to the following conditions: 9 | // 10 | // The above copyright notice and this permission notice shall be included in 11 | // all copies or substantial portions of the Software. 12 | // 13 | // THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR 14 | // IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, 15 | // FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE 16 | // AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER 17 | // LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, 18 | // OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN 19 | // THE SOFTWARE. 20 | 21 | package pool 22 | 23 | import ( 24 | "os" 25 | "sync" 26 | "sync/atomic" 27 | "testing" 28 | "time" 29 | 30 | "github.com/uber/storagetapper/config" 31 | "github.com/uber/storagetapper/log" 32 | "github.com/uber/storagetapper/test" 33 | ) 34 | 35 | var cfg *config.AppConfig 36 | 37 | func waitFor(current *int32, target int, i int, t *testing.T) { 38 | for atomic.LoadInt32(current) != int32(target) && i > 0 { 39 | time.Sleep(time.Millisecond * time.Duration(200)) 40 | i-- 41 | } 42 | 43 | if atomic.LoadInt32(current) != int32(target) { 44 | t.Fatalf("Expected nProcs=%v, current=%v", target, atomic.LoadInt32(current)) 45 | } 46 | } 47 | 48 | func TestBasic(t *testing.T) { 49 | var m sync.Mutex 50 | var nProcs int32 51 | 52 | sig := make(chan bool) 53 | 54 | p := Create() 55 | 56 | if p.NumProcs() != 0 { 57 | t.Fatalf("Initially not zero") 58 | } 59 | 60 | p.Start(2, func() { 61 | m.Lock() 62 | atomic.AddInt32(&nProcs, 1) 63 | log.Debugf("Starting new proc, nProcs=%v", nProcs) 64 | m.Unlock() 65 | for !p.Terminate() { 66 | <-sig 67 | log.Debugf("Woken up") 68 | } 69 | m.Lock() 70 | atomic.AddInt32(&nProcs, -1) 71 | log.Debugf("Terminating proc, nProcs=%v", nProcs) 72 | m.Unlock() 73 | }) 74 | 75 | /* Check that both real number and reported by thread pool equal to expected 76 | * value */ 77 | waitFor(&nProcs, 2, 5, t) 78 | if p.NumProcs() != 2 { 79 | t.Fatalf("numProcs != 2") 80 | } 81 | 82 | p.Adjust(8) 83 | 84 | waitFor(&nProcs, 8, 5, t) 85 | if p.NumProcs() != 8 { 86 | t.Fatalf("numProcs != 8") 87 | } 88 | 89 | p.Adjust(3) 90 | 91 | for i := 0; i < 5; i++ { 92 | sig <- true 93 | } 94 | 95 | waitFor(&nProcs, 3, 5, t) 96 | if p.NumProcs() != 3 { 97 | t.Fatalf("numProcs != 3") 98 | } 99 | 100 | p.Adjust(0) 101 | for i := 0; i < 3; i++ { 102 | sig <- true 103 | } 104 | 105 | waitFor(&nProcs, 0, 5, t) 106 | if p.NumProcs() != 0 { 107 | t.Fatalf("numProcs != 0") 108 | } 109 | } 110 | 111 | func TestMain(m *testing.M) { 112 | cfg = test.LoadConfig() 113 | log.Debugf("Config loaded %v", cfg) 114 | os.Exit(m.Run()) 115 | log.Debugf("Starting shutdown") 116 | } 117 | -------------------------------------------------------------------------------- /pprof_test.go: -------------------------------------------------------------------------------- 1 | // Copyright (c) 2017 Uber Technologies, Inc. 2 | // 3 | // Permission is hereby granted, free of charge, to any person obtaining a copy 4 | // of this software and associated documentation files (the "Software"), to deal 5 | // in the Software without restriction, including without limitation the rights 6 | // to use, copy, modify, merge, publish, distribute, sublicense, and/or sell 7 | // copies of the Software, and to permit persons to whom the Software is 8 | // furnished to do so, subject to the following conditions: 9 | // 10 | // The above copyright notice and this permission notice shall be included in 11 | // all copies or substantial portions of the Software. 12 | // 13 | // THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR 14 | // IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, 15 | // FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE 16 | // AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER 17 | // LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, 18 | // OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN 19 | // THE SOFTWARE. 20 | 21 | package main 22 | 23 | import ( 24 | "context" 25 | "sync" 26 | "testing" 27 | "time" 28 | 29 | "github.com/uber/storagetapper/config" 30 | "github.com/uber/storagetapper/db" 31 | "github.com/uber/storagetapper/shutdown" 32 | "github.com/uber/storagetapper/test" 33 | "github.com/uber/storagetapper/types" 34 | "github.com/uber/storagetapper/util" 35 | ) 36 | 37 | func TestPprofBasic(t *testing.T) { 38 | test.SkipIfNoMySQLAvailable(t) 39 | 40 | conn, err := db.Open(&db.Addr{Host: "localhost", Port: 3306, User: types.TestMySQLUser, Pwd: types.TestMySQLPassword}) 41 | test.CheckFail(err, t) 42 | 43 | test.ExecSQL(conn, t, "DROP DATABASE IF EXISTS "+types.MyDBName) 44 | test.ExecSQL(conn, t, "DROP DATABASE IF EXISTS e2e_test_db1") 45 | test.ExecSQL(conn, t, "RESET MASTER") 46 | 47 | var wg sync.WaitGroup 48 | wg.Add(1) 49 | go func() { 50 | mainLow(config.Get()) 51 | wg.Done() 52 | }() 53 | 54 | /*Wait while it initializes */ 55 | for shutdown.NumProcs() <= 1 { 56 | time.Sleep(time.Millisecond * 500) 57 | } 58 | 59 | ctx, cancel := context.WithTimeout(context.Background(), time.Second*15) 60 | defer cancel() 61 | 62 | _, err = util.HTTPGet(ctx, "http://localhost:7836/debug/pprof/trace?seconds=1") 63 | test.CheckFail(err, t) 64 | _, err = util.HTTPGet(ctx, "http://localhost:7836/debug/pprof/profile?seconds=1") 65 | test.CheckFail(err, t) 66 | _, err = util.HTTPGet(ctx, "http://localhost:7836/debug/pprof/heap") 67 | test.CheckFail(err, t) 68 | 69 | shutdown.Initiate() 70 | shutdown.Wait() 71 | wg.Wait() 72 | } 73 | -------------------------------------------------------------------------------- /schema/alter_schema.go: -------------------------------------------------------------------------------- 1 | // Copyright (c) 2017 Uber Technologies, Inc. 2 | // 3 | // Permission is hereby granted, free of charge, to any person obtaining a copy 4 | // of this software and associated documentation files (the "Software"), to deal 5 | // in the Software without restriction, including without limitation the rights 6 | // to use, copy, modify, merge, publish, distribute, sublicense, and/or sell 7 | // copies of the Software, and to permit persons to whom the Software is 8 | // furnished to do so, subject to the following conditions: 9 | // 10 | // The above copyright notice and this permission notice shall be included in 11 | // all copies or substantial portions of the Software. 12 | // 13 | // THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR 14 | // IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, 15 | // FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE 16 | // AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER 17 | // LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, 18 | // OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN 19 | // THE SOFTWARE. 20 | 21 | package schema 22 | 23 | import ( 24 | "database/sql" 25 | "fmt" 26 | "time" 27 | 28 | "github.com/gofrs/uuid" 29 | "github.com/uber/storagetapper/log" 30 | "github.com/uber/storagetapper/types" 31 | "github.com/uber/storagetapper/util" 32 | ) 33 | 34 | // MutateTable perform alter schema for the given table, using temporary table 35 | // and return structured and raw schema 36 | func MutateTable(sdb *sql.DB, svc string, dbName string, tableName string, alter string, ts *types.TableSchema, rawSchema *string) bool { 37 | 38 | //TODO: Wrap below SQL calls in a transaction 39 | stn, err := uuid.NewV4() 40 | if log.E(err) { 41 | return false 42 | } 43 | tn := time.Now().Format("20060102") + "_" + stn.String() 44 | ftn := "`" + types.MyDBName + "`.`" + tn + "`" 45 | c := fmt.Sprintf("%s_%s_%s", svc, dbName, tableName) 46 | 47 | if log.E(util.ExecSQL(sdb, "CREATE TABLE "+ftn+*rawSchema+" COMMENT='"+c+"'")) { 48 | return false 49 | } 50 | defer func() { 51 | log.E(util.ExecSQL(sdb, "DROP TABLE "+ftn)) 52 | }() 53 | 54 | if log.E(util.ExecSQL(sdb, "ALTER TABLE "+ftn+" "+alter)) { 55 | return false 56 | } 57 | 58 | ct, err := getRawLow(sdb, ftn) 59 | if log.E(err) { 60 | return false 61 | } 62 | 63 | tsn, err := GetColumns(sdb, types.MyDBName, tn) 64 | if log.E(err) { 65 | return false 66 | } 67 | 68 | ts.DBName = dbName 69 | ts.TableName = tableName 70 | ts.Columns = tsn.Columns 71 | *rawSchema = ct 72 | 73 | return true 74 | } 75 | -------------------------------------------------------------------------------- /schema/alter_test.go: -------------------------------------------------------------------------------- 1 | package schema 2 | 3 | import ( 4 | "reflect" 5 | "testing" 6 | 7 | "github.com/uber/storagetapper/db" 8 | "github.com/uber/storagetapper/log" 9 | "github.com/uber/storagetapper/test" 10 | "github.com/uber/storagetapper/types" 11 | ) 12 | 13 | func TestMutateTable(t *testing.T) { 14 | test.SkipIfNoMySQLAvailable(t) 15 | 16 | createTestSchemaTable(t) 17 | 18 | var tblSchema types.TableSchema 19 | 20 | rawSchema, err := GetRaw(&db.Loc{Service: TestSvc, Name: TestDB}, TestTbl, TestInput) 21 | test.CheckFail(err, t) 22 | 23 | if !MutateTable(conn, TestSvc, TestDB, TestTbl, ` ADD f111 BIGINT`, &tblSchema, &rawSchema) { 24 | t.Fatalf("MutateTable failed") 25 | } 26 | 27 | test.ExecSQL(conn, t, `ALTER TABLE `+types.MyDBName+`.`+TestTbl+` ADD f111 BIGINT`) 28 | 29 | tblSchemaRef, err := Get(&db.Loc{Service: TestSvc, Name: TestDB}, TestTbl, TestInput) 30 | test.CheckFail(err, t) 31 | 32 | log.Debugf("%+v", tblSchemaRef) 33 | log.Debugf("%+v", tblSchema) 34 | if !reflect.DeepEqual(tblSchemaRef, &tblSchema) { 35 | t.Fatalf("Wrong mutated schema") 36 | } 37 | 38 | dropTestSchemaTable(t) 39 | } 40 | -------------------------------------------------------------------------------- /schema/mysql_avro_schema_converter.go: -------------------------------------------------------------------------------- 1 | // Copyright (c) 2017 Uber Technologies, Inc. 2 | // 3 | // Permission is hereby granted, free of charge, to any person obtaining a copy 4 | // of this software and associated documentation files (the "Software"), to deal 5 | // in the Software without restriction, including without limitation the rights 6 | // to use, copy, modify, merge, publish, distribute, sublicense, and/or sell 7 | // copies of the Software, and to permit persons to whom the Software is 8 | // furnished to do so, subject to the following conditions: 9 | // 10 | // The above copyright notice and this permission notice shall be included in 11 | // all copies or substantial portions of the Software. 12 | // 13 | // THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR 14 | // IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, 15 | // FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE 16 | // AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER 17 | // LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, 18 | // OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN 19 | // THE SOFTWARE. 20 | 21 | package schema 22 | 23 | import ( 24 | "encoding/json" 25 | "fmt" 26 | "strings" 27 | 28 | "github.com/uber/storagetapper/db" 29 | "github.com/uber/storagetapper/types" 30 | ) 31 | 32 | var ( 33 | namespace = "storagetapper" 34 | ) 35 | 36 | //MySQLToAvroType is the static conversion map from MySQL types to Avro types 37 | var MySQLToAvroType = map[string]types.AvroPrimitiveType{ 38 | "INT": types.AvroINT, 39 | "INTEGER": types.AvroINT, 40 | "TINYINT": types.AvroINT, 41 | "SMALLINT": types.AvroINT, 42 | "MEDIUMINT": types.AvroINT, 43 | "BOOLEAN": types.AvroBOOLEAN, 44 | "BIGINT": types.AvroLONG, 45 | 46 | "FLOAT": types.AvroFLOAT, 47 | "DOUBLE": types.AvroDOUBLE, 48 | "DECIMAL": types.AvroDOUBLE, 49 | "BIT": types.AvroBYTES, 50 | 51 | "CHAR": types.AvroSTRING, 52 | "VARCHAR": types.AvroSTRING, 53 | 54 | //TODO: Confirm correct Avro type for BINARY/VARBINARY types 55 | "BINARY": types.AvroBYTES, 56 | "VARBINARY": types.AvroBYTES, 57 | 58 | "TEXT": types.AvroSTRING, 59 | "TINYTEXT": types.AvroSTRING, 60 | "MEDIUMTEXT": types.AvroSTRING, 61 | "LONGTEXT": types.AvroSTRING, 62 | "JSON": types.AvroSTRING, 63 | 64 | "BLOB": types.AvroBYTES, 65 | "TINYBLOB": types.AvroBYTES, 66 | "MEDIUMBLOB": types.AvroBYTES, 67 | "LONGBLOB": types.AvroBYTES, 68 | 69 | "DATE": types.AvroSTRING, 70 | "DATETIME": types.AvroLONG, 71 | "TIMESTAMP": types.AvroLONG, 72 | "TIME": types.AvroSTRING, 73 | "YEAR": types.AvroINT, 74 | "ENUM": types.AvroSTRING, 75 | "SET": types.AvroSTRING, 76 | //TODO: Add geometry types 77 | } 78 | 79 | // ConvertToAvroFromSchema converts a MySQL schema to an Avro schema 80 | func ConvertToAvroFromSchema(tblSchema *types.TableSchema, formatType string) ([]byte, error) { 81 | avroSchema := &types.AvroSchema{ 82 | Name: fmt.Sprintf("%s_%s", strings.Replace(tblSchema.DBName, "-", "__", -1), strings.Replace(tblSchema.TableName, "-", "__", -1)), 83 | Type: types.AvroRECORD, 84 | Namespace: namespace, 85 | Fields: []types.AvroField{}, 86 | Owner: tblSchema.DBName, 87 | } 88 | 89 | for _, colSchema := range tblSchema.Columns { 90 | avroType := MySQLToAvroType[strings.ToUpper(colSchema.DataType)] 91 | if avroType == "" { 92 | continue 93 | } 94 | if colSchema.Type == types.MySQLBoolean { 95 | avroType = MySQLToAvroType["BOOLEAN"] 96 | } 97 | fieldTypes := []types.AvroPrimitiveType{types.AvroNULL, avroType} 98 | avroField := types.AvroField{ 99 | Name: colSchema.Name, 100 | Type: fieldTypes, 101 | Default: nil, 102 | } 103 | avroSchema.Fields = append(avroSchema.Fields, avroField) 104 | } 105 | 106 | if formatType == "avro" { 107 | fieldTypes := []types.AvroPrimitiveType{types.AvroLONG} 108 | avroField := types.AvroField{ 109 | Name: "ref_key", 110 | Type: fieldTypes, 111 | Default: nil, 112 | } 113 | avroSchema.Fields = append(avroSchema.Fields, avroField) 114 | 115 | fieldTypes = []types.AvroPrimitiveType{types.AvroBYTES} 116 | avroField = types.AvroField{ 117 | Name: "row_key", 118 | Type: fieldTypes, 119 | Default: nil, 120 | } 121 | avroSchema.Fields = append(avroSchema.Fields, avroField) 122 | 123 | fieldTypes = []types.AvroPrimitiveType{types.AvroNULL, types.AvroBOOLEAN} 124 | avroField = types.AvroField{ 125 | Name: "is_deleted", 126 | Type: fieldTypes, 127 | Default: nil, 128 | } 129 | avroSchema.Fields = append(avroSchema.Fields, avroField) 130 | } 131 | 132 | return json.Marshal(avroSchema) 133 | } 134 | 135 | // ConvertToAvro converts a MySQL schema to an Avro schema 136 | func ConvertToAvro(dbl *db.Loc, tableName string, inputType string, formatType string) ([]byte, error) { 137 | tblSchema, err := Get(dbl, tableName, inputType) 138 | if err != nil { 139 | return nil, err 140 | } 141 | return ConvertToAvroFromSchema(tblSchema, formatType) 142 | } 143 | -------------------------------------------------------------------------------- /schema/table_schema.go: -------------------------------------------------------------------------------- 1 | // Copyright (c) 2017 Uber Technologies, Inc. 2 | // 3 | // Permission is hereby granted, free of charge, to any person obtaining a copy 4 | // of this software and associated documentation files (the "Software"), to deal 5 | // in the Software without restriction, including without limitation the rights 6 | // to use, copy, modify, merge, publish, distribute, sublicense, and/or sell 7 | // copies of the Software, and to permit persons to whom the Software is 8 | // furnished to do so, subject to the following conditions: 9 | // 10 | // The above copyright notice and this permission notice shall be included in 11 | // all copies or substantial portions of the Software. 12 | // 13 | // THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR 14 | // IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, 15 | // FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE 16 | // AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER 17 | // LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, 18 | // OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN 19 | // THE SOFTWARE. 20 | 21 | package schema 22 | 23 | import ( 24 | "database/sql" 25 | "fmt" 26 | "regexp" 27 | "strings" 28 | 29 | "github.com/pkg/errors" 30 | "github.com/uber/storagetapper/db" 31 | "github.com/uber/storagetapper/log" 32 | "github.com/uber/storagetapper/types" 33 | ) 34 | 35 | //HasPrimaryKey checks if given table has primary key 36 | func HasPrimaryKey(s *types.TableSchema) bool { 37 | for _, c := range s.Columns { 38 | if c.Key == "PRI" { 39 | return true 40 | } 41 | } 42 | return false 43 | } 44 | 45 | func getRawLow(db *sql.DB, fullTable string) (string, error) { 46 | var ct, unused string 47 | /*FIXME: Can I pass nil here? */ 48 | if err := db.QueryRow("SHOW CREATE TABLE "+fullTable).Scan(&unused, &ct); err != nil { 49 | return "", err 50 | } 51 | 52 | //Cut CONSTRAINTS 53 | var re = regexp.MustCompile(`(?im)^\s*CONSTRAINT.*\n`) 54 | ct = re.ReplaceAllString(ct, ``) 55 | var re1 = regexp.MustCompile(`,\n\)`) // Cut possible remaining comma at the end 56 | ct = re1.ReplaceAllString(ct, ` 57 | )`) //FIXME: Is there a better way to insert \n 58 | 59 | i := strings.Index(ct, "(") 60 | if i == -1 { 61 | return "", errors.New("Broken schema: " + ct) 62 | } 63 | 64 | return ct[i:], nil 65 | } 66 | 67 | // GetRaw returns output of SHOW CREATE TABLE after the "CREATE TABLE xyz (" 68 | // So called "raw" schema 69 | func GetRaw(dbl *db.Loc, fullTable string, inputType string) (string, error) { 70 | conn, err := db.OpenService(dbl, "", inputType) 71 | if err != nil { 72 | return "", err 73 | } 74 | defer func() { log.E(conn.Close()) }() 75 | return getRawLow(conn, fullTable) 76 | } 77 | 78 | // Get loads structured schema for "table", from master DB, identified by dbl 79 | func Get(dbl *db.Loc, table string, inputType string) (*types.TableSchema, error) { 80 | conn, err := db.OpenService(dbl, "information_schema", inputType) 81 | if err != nil { 82 | return nil, err 83 | } 84 | defer func() { log.E(conn.Close()) }() 85 | return GetColumns(conn, dbl.Name, table) 86 | } 87 | 88 | // ParseColumnInfo reads parses column information into table schema 89 | func ParseColumnInfo(rows *sql.Rows, dbName, table string) (*types.TableSchema, error) { 90 | tableSchema := types.TableSchema{DBName: dbName, TableName: table, Columns: []types.ColumnSchema{}} 91 | 92 | rowsCount := 0 93 | for rows.Next() { 94 | cs := types.ColumnSchema{} 95 | err := rows.Scan(&cs.Name, &cs.OrdinalPosition, &cs.IsNullable, 96 | &cs.DataType, &cs.CharacterMaximumLength, &cs.NumericPrecision, 97 | &cs.NumericScale, &cs.Type, &cs.Key) 98 | if err != nil { 99 | log.E(errors.Wrap(err, fmt.Sprintf("Error scanning table schema query result for %s.%s", 100 | dbName, table))) 101 | return nil, err 102 | } 103 | tableSchema.Columns = append(tableSchema.Columns, cs) 104 | rowsCount++ 105 | } 106 | if err := rows.Err(); err != nil { 107 | log.F(err) 108 | } 109 | if rowsCount == 0 { 110 | return &tableSchema, fmt.Errorf("no schema columns for table %s.%s. check grants", dbName, table) 111 | } 112 | 113 | log.Debugf("Got schema from state for '%v.%v' = '%+v'", dbName, table, tableSchema) 114 | return &tableSchema, nil 115 | } 116 | 117 | // GetColumns reads structured schema from information_schema for table from given connection 118 | func GetColumns(conn *sql.DB, dbName string, tableName string) (*types.TableSchema, error) { 119 | query := "SELECT COLUMN_NAME, ORDINAL_POSITION, IS_NULLABLE, DATA_TYPE, " + 120 | "CHARACTER_MAXIMUM_LENGTH, NUMERIC_PRECISION, NUMERIC_SCALE, COLUMN_TYPE, " + 121 | "COLUMN_KEY FROM information_schema.columns WHERE TABLE_SCHEMA = ? AND TABLE_NAME = ? " + 122 | "ORDER BY ORDINAL_POSITION" 123 | 124 | log.Debugf("%v %v %v", query, dbName, tableName) 125 | 126 | rows, err := conn.Query(query, dbName, tableName) 127 | if log.E(errors.Wrap(err, fmt.Sprintf("Error fetching table schema for %s.%s", dbName, tableName))) { 128 | return nil, err 129 | } 130 | defer func() { log.E(rows.Close()) }() 131 | 132 | return ParseColumnInfo(rows, dbName, tableName) 133 | } 134 | -------------------------------------------------------------------------------- /scripts/docker_test_entry.sh: -------------------------------------------------------------------------------- 1 | #!/bin/bash 2 | 3 | set -ex 4 | 5 | export GOPATH=$HOME/gopath 6 | export PATH=$HOME/gopath/bin:/usr/local/go/bin:$PATH 7 | export STORAGETAPPER_ENVIRONMENT="test" 8 | 9 | /bin/sh scripts/prepare_test_env.sh 10 | 11 | P=$GOPATH/src/github.com/uber 12 | mkdir -p "$P" 13 | cp -ap /storagetapper "$P" 14 | cd "$P"/storagetapper 15 | 16 | STORAGTAPPER_CONFIG_DIR=$(pwd)/config 17 | export STORAGTAPPER_CONFIG_DIR 18 | 19 | export GO111MODULE=on 20 | make test 21 | -------------------------------------------------------------------------------- /scripts/install_deps.sh: -------------------------------------------------------------------------------- 1 | #!/bin/sh 2 | 3 | set -ex 4 | 5 | /bin/sh scripts/install_go.sh 6 | /bin/sh scripts/install_mysql.sh 7 | /bin/sh scripts/install_kafka.sh 8 | /bin/sh scripts/install_hadoop.sh 9 | /bin/sh scripts/install_sql.sh 10 | /bin/sh scripts/install_hive.sh 11 | -------------------------------------------------------------------------------- /scripts/install_go.sh: -------------------------------------------------------------------------------- 1 | #!/bin/sh 2 | 3 | set -ex 4 | 5 | GOVERSION=1.17.3 6 | [ -f go$GOVERSION.linux-amd64.tar.gz ] || wget https://dl.google.com/go/go$GOVERSION.linux-amd64.tar.gz 7 | echo "550f9845451c0c94be679faf116291e7807a8d78b43149f9506c1b15eb89008c go$GOVERSION.linux-amd64.tar.gz" | sha256sum -c - 8 | sudo tar -xzf go$GOVERSION.linux-amd64.tar.gz -C /usr/local 9 | 10 | export GOPATH=$HOME/gopath 11 | export PATH=/usr/local/go/bin:$PATH 12 | export GOROOT=/usr/local/go 13 | curl -sfL https://raw.githubusercontent.com/golangci/golangci-lint/master/install.sh| sh -s -- -b "$(go env GOPATH)/bin" v1.43.0 14 | go install github.com/tinylib/msgp@v1.1.6 15 | -------------------------------------------------------------------------------- /scripts/install_hadoop.sh: -------------------------------------------------------------------------------- 1 | #!/bin/bash 2 | 3 | set -ex 4 | 5 | NAME=hadoop 6 | DIR=/home/$NAME 7 | VERSION=3.3.1 8 | SHA512="2fd0bf74852c797dc864f373ec82ffaa1e98706b309b30d1effa91ac399b477e1accc1ee74d4ccbb1db7da1c5c541b72e4a834f131a99f2814b030fbd043df66" 9 | 10 | sudo useradd $NAME -m || [ $? -eq 9 ] 11 | cd $DIR 12 | [ -f hadoop-$VERSION.tar.gz ] || sudo -H -u $NAME wget "https://mirrors.ocf.berkeley.edu/apache/hadoop/common/hadoop-$VERSION/hadoop-$VERSION.tar.gz" -O hadoop-$VERSION.tar.gz 13 | echo "$SHA512 hadoop-$VERSION.tar.gz" | sha512sum -c 14 | sudo -H -u $NAME tar -xzf hadoop-$VERSION.tar.gz --strip 1 15 | 16 | cat << 'EOF' | sudo -H -u $NAME tee $DIR/etc/hadoop/core-site.xml 17 | 18 | 19 | fs.defaultFS 20 | hdfs://localhost:9000 21 | 22 | 23 | EOF 24 | 25 | cat << 'EOF' | sudo -H -u $NAME tee $DIR/etc/hadoop/hdfs-site.xml 26 | 27 | 28 | dfs.replication 29 | 1 30 | 31 | 32 | dfs.datanode.handler.count 33 | 20 34 | 35 | 36 | dfs.blocksize 37 | 1048576 38 | 39 | 40 | EOF 41 | 42 | sudo -H -u $NAME /bin/bash < ~/.ssh/authorized_keys 47 | chmod 0600 ~/.ssh/authorized_keys 48 | ssh-keyscan -H localhost > ~/.ssh/known_hosts 49 | ssh-keyscan -H 0.0.0.0 >> ~/.ssh/known_hosts 50 | rm -rf /tmp/hadoop-hadoop 51 | fi 52 | bin/hdfs namenode -format -force 53 | if [ -z "$DOCKER_BUILD" ]; then 54 | (sbin/start-dfs.sh) & 55 | PID=\$! 56 | wait \$PID 57 | bin/hdfs dfs -mkdir -p /user/$USER 58 | bin/hdfs dfs -chown -R $USER:$(id -g -n) /user/$USER 59 | fi 60 | EOF 61 | -------------------------------------------------------------------------------- /scripts/install_hive.sh: -------------------------------------------------------------------------------- 1 | #!/bin/bash 2 | 3 | set -ex 4 | 5 | NAME=hadoop 6 | DIR=/home/$NAME 7 | VERSION=2.3.9 8 | 9 | sudo apt-get install --force-yes -y default-jre wget thrift-compiler 10 | sudo useradd $NAME -m || [ $? -eq 9 ] 11 | cd $DIR 12 | [ -f apache-hive-${VERSION}-bin.tar.gz ] || sudo -H -u $NAME wget "http://apache.cs.utah.edu/hive/hive-${VERSION}/apache-hive-${VERSION}-bin.tar.gz" -O apache-hive-${VERSION}-bin.tar.gz 13 | echo "0ad229e20e30d259d20ea6ede5c8fd4000c5844dac9e40e7f35128e0cdc55cc4f71046582db0a6c63effada4599952f8b4b10f2c394c042657aba134bff9c598 apache-hive-${VERSION}-bin.tar.gz" | sha512sum -c 14 | sudo -H -u $NAME /bin/bash <> $DIR/kafka-$KAFKA_PORT/config/server.properties 35 | 36 | mkdir -p $KAFKA_DATADIR 37 | sed -i -e "s#log.dirs=.*#log.dirs=$KAFKA_DATADIR#g" $DIR/kafka-$KAFKA_PORT/config/server.properties 38 | 39 | sed -i -e "s#dataDir=.*#dataDir=$ZK_DATADIR#g" -e "s/clientPort=.*/clientPort=$ZK_PORT/g" $DIR/kafka-$KAFKA_PORT/config/zookeeper.properties 40 | echo -e "\\ninitLimit=10\\nsyncLimit=5\\nserver.1=localhost:2888:3888\\nserver.2=localhost:2889:3889\\nserver.3=localhost:2890:3890\\n" >> $DIR/kafka-$KAFKA_PORT/config/zookeeper.properties 41 | 42 | mkdir -p $ZK_DATADIR 43 | 44 | echo $i > $DIR/zookeeper-$ZK_PORT/myid 45 | EOF 46 | done 47 | ) 48 | 49 | if [ -z "$DOCKER_BUILD" ]; then 50 | sudo -H -u kafka /bin/bash scripts/start_kafka.sh 51 | fi 52 | -------------------------------------------------------------------------------- /scripts/install_mysql.sh: -------------------------------------------------------------------------------- 1 | #!/bin/sh 2 | 3 | set -ex 4 | 5 | #Latest version and md5sum can be found at https://dev.mysql.com/downloads/repo/apt/ 6 | M_APT_VER="0.8.14-1" 7 | sudo mv /var/lib/mysql /var/lib/mysql."$(date +%y%m%d%H%M%S)" || true 8 | [ -f mysql-apt-config_${M_APT_VER}_all.deb ] || wget https://dev.mysql.com/get/mysql-apt-config_${M_APT_VER}_all.deb 9 | echo "5cc94c7720fcd3124449b3e789441b98 mysql-apt-config_${M_APT_VER}_all.deb" | md5sum -c - 10 | sudo MYSQL_SERVER_VERSION="mysql-8.0" DEBIAN_FRONTEND="noninteractive" /bin/sh -c "dpkg -i mysql-apt-config_${M_APT_VER}_all.deb && \ 11 | sudo -E apt-get update && \ 12 | sudo -E apt-get install -y --force-yes git make mysql-community-server" 13 | sudo /usr/bin/mysqladmin shutdown || true 14 | printf "[mysqld]\nserver-id=1\nbinlog-format=ROW\ngtid_mode=ON\nenforce-gtid-consistency\nlog_bin=/var/log/mysql/mysql-bin.log\nlog_slave_updates=1\ninnodb_flush_log_at_trx_commit=0\ninnodb_flush_log_at_timeout=30"|sudo tee -a /etc/mysql/my.cnf 15 | sudo /usr/share/mysql-8.0/mysql-systemd-start pre 16 | sudo /usr/bin/mysqld_safe --skip-syslog & 17 | while ! sudo /usr/bin/mysqladmin ping; do sleep 1; done 18 | sudo mysql -e "DROP USER IF EXISTS 'storagetapper';CREATE USER 'storagetapper' IDENTIFIED BY 'storagetapper';GRANT ALL ON *.* TO 'storagetapper'@'%';FLUSH PRIVILEGES" 19 | if [ -n "$DOCKER_BUILD" ]; then 20 | sudo /usr/bin/mysqladmin shutdown || true 21 | fi 22 | -------------------------------------------------------------------------------- /scripts/install_sql.sh: -------------------------------------------------------------------------------- 1 | #!/bin/bash 2 | 3 | set -ex 4 | 5 | sudo /bin/bash -c "echo 'deb http://repo.yandex.ru/clickhouse/deb/stable/ main/'>/etc/apt/sources.list.d/clickhouse.list" 6 | sudo apt-key adv --keyserver keyserver.ubuntu.com --recv E0C56BD4 7 | sudo apt-get update 8 | sudo DEBIAN_FRONTEND=noninteractive apt-get install -y clickhouse-client clickhouse-server 9 | 10 | #Avoid conflict with Hadoop default port 11 | sudo sed -i 's+9000+9500+g' /etc/clickhouse-server/config.xml 12 | 13 | sudo apt-get install --force-yes -y postgresql shellcheck 14 | 15 | #echo "host all postgres 127.0.0.1/32 trust" | sudo -H tee -a `find /etc/postgresql -name pg_hba.conf` 16 | find /etc/postgresql -name pg_hba.conf -exec sudo -H sed -i -e 's/md5/trust/g' -e 's/peer/trust/g' {} \; 17 | 18 | sudo /etc/init.d/postgresql restart 19 | -------------------------------------------------------------------------------- /scripts/prepare_test_env.sh: -------------------------------------------------------------------------------- 1 | #!/bin/bash 2 | 3 | set -ex 4 | 5 | sudo /usr/share/mysql-8.0/mysql-systemd-start pre 6 | sudo /usr/bin/mysqld_safe --skip-syslog & 7 | while ! /usr/bin/mysqladmin ping; do sleep 1; done 8 | /etc/init.d/postgresql start 9 | /etc/init.d/clickhouse-server start 10 | 11 | sudo -H -u kafka /bin/bash scripts/start_kafka.sh 12 | 13 | # Now startup Hadoop 14 | service ssh start 15 | sudo -u hadoop /bin/bash < ~/.ssh/known_hosts 17 | ssh-keyscan -H 0.0.0.0 >> ~/.ssh/known_hosts 18 | cd /home/hadoop 19 | (sbin/start-dfs.sh) & 20 | PID=\$! 21 | wait \$PID 22 | bin/hdfs dfs -mkdir -p /user/$USER 23 | bin/hdfs dfs -chown -R $USER:$(id -g -n) /user/$USER 24 | EOF 25 | 26 | 27 | sudo -H -u hadoop /bin/bash </dev/null || exit 1 18 | if [ -f profile.out ]; then 19 | cat profile.out >> coverage.out #combine coverage report for codecov.io 20 | rm profile.out 21 | fi 22 | done 23 | -------------------------------------------------------------------------------- /scripts/run_lints.sh: -------------------------------------------------------------------------------- 1 | #!/bin/sh 2 | 3 | set -e 4 | 5 | for i in "$@"; do 6 | CGO_ENABLED=0 golangci-lint run --skip-files format_gen --disable-all \ 7 | -Egofmt \ 8 | -Egovet \ 9 | -Erevive \ 10 | -Egoimports \ 11 | -Eineffassign \ 12 | -Eerrcheck \ 13 | -Edeadcode \ 14 | -Emisspell \ 15 | -Egocyclo \ 16 | -Estaticcheck \ 17 | -Egosimple \ 18 | -Estructcheck \ 19 | -Etypecheck \ 20 | -Eunused \ 21 | -Evarcheck \ 22 | -Eunconvert \ 23 | -Eprealloc \ 24 | -Estylecheck \ 25 | "$i" || exit 1 26 | printf "ok\t%s\n" "$i" 27 | done 28 | -------------------------------------------------------------------------------- /scripts/run_tests.sh: -------------------------------------------------------------------------------- 1 | #!/bin/bash 2 | 3 | STORAGETAPPER_CONFIG_DIR=$(pwd)/config 4 | export STORAGETAPPER_CONFIG_DIR 5 | 6 | if [ -z "$STORAGETAPPER_ENVIRONMENT" ]; then 7 | export STORAGETAPPER_ENVIRONMENT=development 8 | fi 9 | 10 | TIMEOUT=600s 11 | 12 | export GOTRACEBACK="crash" #produce core file on panic 13 | 14 | if [ -z "$NOCOVER" ]; then 15 | COVER="-coverprofile=profile.out -covermode=atomic" 16 | fi 17 | 18 | #FIXME: Because of the shared state in database tests can't be run in parallel 19 | CMD="go test -race $COVER -test.timeout $TIMEOUT $TEST_PARAM" 20 | 21 | for i in "$@"; do 22 | $CMD "$i" || exit 1 23 | if [ -f profile.out ]; then 24 | cat profile.out >> coverage.out #combine coverage report for codecov.io 25 | rm profile.out 26 | fi 27 | done 28 | -------------------------------------------------------------------------------- /scripts/start_kafka.sh: -------------------------------------------------------------------------------- 1 | #!/bin/bash 2 | 3 | set -ex 4 | 5 | DIR=/home/kafka 6 | TIMEOUT=120 7 | 8 | for i in 1 2 3; do 9 | KAFKA_PORT=$((i + 9090)) 10 | 11 | KAFKA_HEAP_OPTS="-Xmx192m" $DIR/kafka-"$KAFKA_PORT"/bin/zookeeper-server-start.sh -daemon $DIR/kafka-"$KAFKA_PORT"/config/zookeeper.properties 12 | done 13 | 14 | for i in 1 2 3; do 15 | j=0 16 | while ! nc -z localhost $((2180+i)) && [ "$((j++))" -lt "$TIMEOUT" ]; do echo "Waiting %i"; sleep 1; done 17 | if [ "$j" -eq "$TIMEOUT" ]; then 18 | echo "Timeout waiting zookeeper($i) to start" 19 | cat /home/kafka/kafka-$((9090+i))/logs/zookeeper.out 20 | exit 1 21 | fi 22 | done 23 | 24 | sleep 1 # give time zookeeper to initialize 25 | 26 | # Launch and wait for Kafka 27 | for i in 1 2 3; do 28 | KAFKA_PORT=$((i + 9090)) 29 | 30 | KAFKA_HEAP_OPTS="-Xmx320m" $DIR/kafka-"$KAFKA_PORT"/bin/kafka-server-start.sh -daemon $DIR/kafka-"$KAFKA_PORT"/config/server.properties 31 | done 32 | 33 | for i in 1 2 3; do 34 | j=0 35 | while ! nc -z localhost $((9090+i)) && [ "$((j++))" -lt "$TIMEOUT" ]; do echo "Waiting"; sleep 1; done 36 | if [ "$j" -eq "$TIMEOUT" ]; then 37 | echo "Timeout waiting Kafka ($i) to start" 38 | cat /home/kafka/kafka-$((9090+i))/logs/server.log 39 | exit 1 40 | fi 41 | done 42 | 43 | sleep 7 # give time kafka to initialize 44 | -------------------------------------------------------------------------------- /scripts/stop_kafka.sh: -------------------------------------------------------------------------------- 1 | #!/bin/bash 2 | 3 | set -x 4 | 5 | DIR=/home/kafka 6 | 7 | for i in 1 2 3; do 8 | KAFKA_PORT=$((i + 9090)) 9 | $DIR/kafka-"$KAFKA_PORT"/bin/kafka-server-stop.sh 10 | done 11 | 12 | for i in 1 2 3; do 13 | KAFKA_PORT=$((i + 9090)) 14 | $DIR/kafka-$KAFKA_PORT/bin/zookeeper-server-stop.sh 15 | done 16 | -------------------------------------------------------------------------------- /scripts/workflow_example.sh: -------------------------------------------------------------------------------- 1 | #!/bin/sh 2 | 3 | set -ex 4 | 5 | go build 6 | 7 | sql() { 8 | db=$1 9 | shift 10 | mysql -uroot "$db" -e "$@" 11 | } 12 | 13 | KAFKA_RETENTION_TIME=20 14 | 15 | TEST_TOPIC=hp-ex_svc1-ex_db1-ex_table1 16 | KPATH=/home/kafka/bin 17 | KPARAM="--zookeeper localhost --topic $TEST_TOPIC" 18 | 19 | $KPATH/kafka-topics.sh "$KPARAM" --alter --config retention.ms=$((KAFKA_RETENTION_TIME * 1000)) 20 | $KPATH/kafka-topics.sh "$KPARAM" --describe 21 | 22 | sql "" "DROP DATABASE IF EXISTS ex_db1" 23 | sql "" "RESET MASTER" 24 | 25 | sql "" "CREATE DATABASE ex_db1" 26 | sql "ex_db1" "CREATE TABLE ex_table1(f1 int not null primary key, ts TIMESTAMP, f3 int not null default 0)" 27 | 28 | for i in $(seq 101 110); do 29 | sql "ex_db1" "INSERT INTO ex_table1(f1) VALUES ($i)" 30 | done 31 | 32 | ./storagetapper & 33 | TPID=$! 34 | trap 'kill $TPID; exit 1' 1 2 15 #SIGHUP SIGINT SIGTERM 35 | sleep 2 36 | 37 | curl --data '{"cmd" : "add", "name" : "ex_cluster1", "host" : "localhost", "port" : 3306, "user" : "root", "pw" : ""}' http://localhost:7836/cluster 38 | curl --data '{"cmd" : "add", "cluster" : "ex_cluster1", "service" : "ex_svc1", "db":"ex_db1", "table":"ex_table1"}' http://localhost:7836/table 39 | 40 | sleep 12 41 | 42 | for i in $(seq 1 10); do 43 | sql "ex_db1" "INSERT INTO ex_table1(f1) VALUES ($i)" 44 | done 45 | 46 | sql "ex_db1" "ALTER TABLE ex_table1 ADD f2 varchar(32)" 47 | 48 | for i in $(seq 11 30); do 49 | sql "ex_db1" "INSERT INTO ex_table1(f1,f2) VALUES ($i, CONCAT('bbb', $i))" 50 | done 51 | 52 | sql "ex_db1" "ALTER TABLE ex_table1 DROP f2" 53 | 54 | for i in $(seq 101 110); do 55 | sql "ex_db1" "UPDATE ex_table1 SET f3=f3+20 WHERE f1>100 AND f1<111" 56 | done 57 | 58 | sleep 4 59 | 60 | curl --data '{"cmd" : "del", "name" : "ex_cluster1" }' http://localhost:7836/cluster 61 | curl --data '{"cmd" : "del", "cluster" : "ex_cluster1", "service" : "ex_svc1", "db":"ex_db1", "table":"ex_table1"}' http://localhost:7836/table 62 | 63 | kill $TPID 64 | 65 | $KPATH/kafka-topics.sh "$KPARAM" --describe 66 | $KPATH/kafka-topics.sh --list --zookeeper localhost:2181 67 | $KPATH/kafka-console-consumer.sh "$KPARAM" --max-messages 50 --from-beginning 68 | 69 | date 70 | echo "Wait for $KAFKA_RETENTION_TIME secs before running next test" 71 | -------------------------------------------------------------------------------- /server/config.go: -------------------------------------------------------------------------------- 1 | // Copyright (c) 2017 Uber Technologies, Inc. 2 | // 3 | // Permission is hereby granted, free of charge, to any person obtaining a copy 4 | // of this software and associated documentation files (the "Software"), to deal 5 | // in the Software without restriction, including without limitation the rights 6 | // to use, copy, modify, merge, publish, distribute, sublicense, and/or sell 7 | // copies of the Software, and to permit persons to whom the Software is 8 | // furnished to do so, subject to the following conditions: 9 | // 10 | // The above copyright notice and this permission notice shall be included in 11 | // all copies or substantial portions of the Software. 12 | // 13 | // THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR 14 | // IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, 15 | // FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE 16 | // AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER 17 | // LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, 18 | // OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN 19 | // THE SOFTWARE. 20 | 21 | package server 22 | 23 | import ( 24 | "encoding/json" 25 | "mime" 26 | "net/http" 27 | 28 | yaml "gopkg.in/yaml.v2" 29 | 30 | "errors" 31 | 32 | "github.com/uber/storagetapper/config" 33 | "github.com/uber/storagetapper/log" 34 | ) 35 | 36 | type configReq struct { 37 | Cmd string 38 | Body string 39 | } 40 | 41 | func configCmd(w http.ResponseWriter, r *http.Request) { 42 | var err error 43 | s := configReq{} 44 | g := config.AppConfigODS{} 45 | ct, _, _ := mime.ParseMediaType(r.Header.Get("Content-Type")) 46 | if r.FormValue("cmd") == "get" { 47 | var b []byte 48 | b, err = yaml.Marshal(config.Get().AppConfigODS) 49 | if err == nil { 50 | w.Header().Set("Content-Type", "application/x-yaml") 51 | _, _ = w.Write(b) 52 | } 53 | } else { // set 54 | switch { 55 | case ct == "application/x-www-form-urlencoded", ct == "multipart/form-data", ct == "": 56 | if len(r.FormValue("body")) == 0 { 57 | err = errors.New("config body is empty") 58 | } else { 59 | err = yaml.Unmarshal([]byte(r.FormValue("body")), &g) 60 | } 61 | case ct == "application/x-yaml", ct == "text/vnd.yaml": 62 | if r.Body == nil { 63 | err = errors.New("config body is empty") 64 | } else { 65 | err = yaml.NewDecoder(r.Body).Decode(&g) 66 | } 67 | case ct == "application/json": 68 | if r.Body == nil { 69 | err = errors.New("config body is empty") 70 | } else { 71 | err = json.NewDecoder(r.Body).Decode(&g) 72 | } 73 | default: 74 | code := http.StatusUnsupportedMediaType 75 | http.Error(w, http.StatusText(code), code) 76 | return 77 | } 78 | if err == nil { 79 | if err = config.Set(&g); err == nil { 80 | err = config.Save() 81 | } 82 | } 83 | } 84 | if err != nil { 85 | log.Errorf("Config http: cmd=%v, body='%v', error=%v", s.Cmd, g, err) 86 | http.Error(w, err.Error(), http.StatusInternalServerError) 87 | } 88 | } 89 | -------------------------------------------------------------------------------- /server/config_test.go: -------------------------------------------------------------------------------- 1 | // Copyright (c) 2017 Uber Technologies, Inc. 2 | // 3 | // Permission is hereby granted, free of charge, to any person obtaining a copy 4 | // of this software and associated documentation files (the "Software"), to deal 5 | // in the Software without restriction, including without limitation the rights 6 | // to use, copy, modify, merge, publish, distribute, sublicense, and/or sell 7 | // copies of the Software, and to permit persons to whom the Software is 8 | // furnished to do so, subject to the following conditions: 9 | // 10 | // The above copyright notice and this permission notice shall be included in 11 | // all copies or substantial portions of the Software. 12 | // 13 | // THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR 14 | // IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, 15 | // FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE 16 | // AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER 17 | // LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, 18 | // OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN 19 | // THE SOFTWARE. 20 | 21 | package server 22 | 23 | import ( 24 | "net/http" 25 | "net/http/httptest" 26 | "net/url" 27 | "strings" 28 | "testing" 29 | 30 | yaml "gopkg.in/yaml.v2" 31 | 32 | "github.com/uber/storagetapper/config" 33 | "github.com/uber/storagetapper/log" 34 | "github.com/uber/storagetapper/state" 35 | "github.com/uber/storagetapper/test" 36 | "github.com/uber/storagetapper/types" 37 | ) 38 | 39 | //TODO: More tests 40 | 41 | func configsTableInit(t *testing.T) { 42 | conn := state.GetDB() 43 | if conn == nil { 44 | t.FailNow() 45 | } 46 | _, err := conn.Exec("TRUNCATE TABLE " + types.MyDBName + ".config") 47 | if err != nil { 48 | log.Errorf("Error truncating table: %v", err) 49 | } 50 | } 51 | 52 | /* 53 | func configJSONRequest(cmd configReq, code int, t *testing.T) { 54 | body, _ := json.Marshal(cmd) 55 | req, err := http.NewRequest("POST", "/config", bytes.NewReader(body)) 56 | req.Header.Add("Content-Type", "application/json") 57 | test.Assert(t, err == nil, "Failed: %v", err) 58 | res := httptest.NewRecorder() 59 | configCmd(res, req) 60 | test.Assert(t, res.Code == code, "Not OK: code=%v", res.Code) 61 | } 62 | */ 63 | 64 | func configFormRequest(typ string, cmd configReq, code int, t *testing.T) *httptest.ResponseRecorder { 65 | body := url.Values{} 66 | body.Add("cmd", cmd.Cmd) 67 | body.Add("body", cmd.Body) 68 | req, err := http.NewRequest("GET", "/config?"+body.Encode(), nil) 69 | if typ == "POST" { 70 | req, err = http.NewRequest("POST", "/config", strings.NewReader(body.Encode())) 71 | } 72 | test.Assert(t, err == nil, "Failed: %v", err) 73 | req.Header.Add("Content-Type", "application/x-www-form-urlencoded") 74 | res := httptest.NewRecorder() 75 | configCmd(res, req) 76 | test.Assert(t, res.Code == code, "Not OK: code=%v", res.Code) 77 | 78 | return res 79 | } 80 | 81 | /* 82 | func configRequest(cmd configReq, code int, t *testing.T) { 83 | configJSONRequest(cmd, code, t) 84 | configFormRequest("POST", cmd, code, t) 85 | configFormRequest("GET", cmd, code, t) 86 | } 87 | */ 88 | 89 | func TestConfigCommands(t *testing.T) { 90 | c := config.Get() 91 | configsTableInit(t) 92 | defer func() { 93 | configsTableInit(t) 94 | err := config.Set(&c.AppConfigODS) 95 | log.E(err) 96 | }() 97 | get := configReq{ 98 | Cmd: "get", 99 | } 100 | resp := configFormRequest("GET", get, http.StatusOK, t) 101 | g := &config.AppConfigODS{} 102 | err := yaml.Unmarshal(resp.Body.Bytes(), &g) 103 | test.CheckFail(err, t) 104 | test.Assert(t, g.LogType == "zap", "expected default log_type is zap. got %v", g.LogType) 105 | g.LogType = "otherlogtype" 106 | b, err := yaml.Marshal(g) 107 | test.CheckFail(err, t) 108 | set := configReq{ 109 | Cmd: "set", 110 | Body: string(b), 111 | } 112 | configFormRequest("POST", set, http.StatusOK, t) 113 | g.LogType = "" 114 | resp = configFormRequest("GET", get, http.StatusOK, t) 115 | err = yaml.Unmarshal(resp.Body.Bytes(), &g) 116 | test.CheckFail(err, t) 117 | test.Assert(t, g.LogType == "otherlogtype", "expected default log_type is uber") 118 | } 119 | -------------------------------------------------------------------------------- /server/server.go: -------------------------------------------------------------------------------- 1 | // Copyright (c) 2017 Uber Technologies, Inc. 2 | // 3 | // Permission is hereby granted, free of charge, to any person obtaining a copy 4 | // of this software and associated documentation files (the "Software"), to deal 5 | // in the Software without restriction, including without limitation the rights 6 | // to use, copy, modify, merge, publish, distribute, sublicense, and/or sell 7 | // copies of the Software, and to permit persons to whom the Software is 8 | // furnished to do so, subject to the following conditions: 9 | // 10 | // The above copyright notice and this permission notice shall be included in 11 | // all copies or substantial portions of the Software. 12 | // 13 | // THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR 14 | // IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, 15 | // FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE 16 | // AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER 17 | // LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, 18 | // OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN 19 | // THE SOFTWARE. 20 | 21 | package server 22 | 23 | import ( 24 | "fmt" 25 | "net/http" 26 | "sync" 27 | 28 | "github.com/uber/storagetapper/log" 29 | "github.com/uber/storagetapper/state" 30 | 31 | "golang.org/x/net/context" 32 | ) 33 | 34 | var server *http.Server 35 | var mutex = sync.Mutex{} 36 | 37 | func init() { 38 | http.HandleFunc("/health", healthCheck) 39 | http.HandleFunc("/schema", schemaCmd) 40 | http.HandleFunc("/cluster", clusterInfoCmd) 41 | http.HandleFunc("/table", tableCmd) 42 | http.HandleFunc("/config", configCmd) 43 | http.HandleFunc("/", indexCmd) 44 | } 45 | 46 | //StartHTTPServer starts listening and serving traffic on configured port and sets up http routes. 47 | func StartHTTPServer(port int) { 48 | state.EmitRegisteredTablesCount() 49 | mutex.Lock() 50 | 51 | server = &http.Server{Addr: fmt.Sprintf(":%d", port), Handler: nil} 52 | log.Debugf("HTTP server is listening on %v port", port) 53 | 54 | mutex.Unlock() 55 | 56 | err := server.ListenAndServe() 57 | if err != nil && err != http.ErrServerClosed { 58 | log.E(err) 59 | } 60 | } 61 | 62 | //healthCheck handles call to the health check endpoint 63 | func healthCheck(w http.ResponseWriter, r *http.Request) { 64 | w.Header().Add("Content-Type", "text/plain") 65 | w.WriteHeader(http.StatusOK) 66 | if _, err := w.Write([]byte("OK")); err != nil { 67 | log.Errorf("Health check failed: %s\n", err) 68 | } 69 | } 70 | 71 | //Shutdown gracefully stops the server 72 | func Shutdown() { 73 | mutex.Lock() 74 | defer mutex.Unlock() 75 | if server == nil { 76 | return 77 | } 78 | ctx, cancel := context.WithCancel(context.Background()) 79 | defer cancel() 80 | log.E(server.Shutdown(ctx)) 81 | server = nil 82 | } 83 | -------------------------------------------------------------------------------- /shutdown/shutdown.go: -------------------------------------------------------------------------------- 1 | // Copyright (c) 2017 Uber Technologies, Inc. 2 | // 3 | // Permission is hereby granted, free of charge, to any person obtaining a copy 4 | // of this software and associated documentation files (the "Software"), to deal 5 | // in the Software without restriction, including without limitation the rights 6 | // to use, copy, modify, merge, publish, distribute, sublicense, and/or sell 7 | // copies of the Software, and to permit persons to whom the Software is 8 | // furnished to do so, subject to the following conditions: 9 | // 10 | // The above copyright notice and this permission notice shall be included in 11 | // all copies or substantial portions of the Software. 12 | // 13 | // THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR 14 | // IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, 15 | // FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE 16 | // AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER 17 | // LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, 18 | // OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN 19 | // THE SOFTWARE. 20 | 21 | /*Package shutdown provides a mechanism for graceful shutdown of multithreaded programs. 22 | It combines functionality of WaitGroups and context.WithCancel, to account 23 | number running threads and notify threads waiting in the blocking calls. 24 | 25 | Setup/Wait allowed to be called sequentially only 26 | Register can be called only from the thread where Setup/Wait called or from the 27 | thread for which register has been called already. 28 | */ 29 | package shutdown 30 | 31 | import ( 32 | "os" 33 | "os/signal" 34 | "sync" 35 | "sync/atomic" 36 | "syscall" 37 | 38 | "golang.org/x/net/context" //"context" 39 | ) 40 | 41 | var group sync.WaitGroup 42 | var flag int32 43 | 44 | /*Context will be canceled when Initiate is called*/ 45 | var Context context.Context 46 | var cancel context.CancelFunc 47 | var numThreads int32 48 | 49 | /*Initiate notifies threads of program's intent to terminate*/ 50 | //Should be called by one thread only 51 | func Initiate() { 52 | if atomic.LoadInt32(&flag) == 0 { 53 | atomic.StoreInt32(&flag, 1) 54 | cancel() 55 | } 56 | } 57 | 58 | /*Initiated is used by threads to check if program is being terminated*/ 59 | func Initiated() bool { 60 | return atomic.LoadInt32(&flag) != 0 61 | } 62 | 63 | /*InitiatedCh is used by threads to receive terminate notification from channel*/ 64 | func InitiatedCh() <-chan struct{} { 65 | return Context.Done() 66 | } 67 | 68 | //Register thread so that Wait function will wait for it termination 69 | //Register should be called before starting threads routine 70 | func Register(i int32) { 71 | atomic.AddInt32(&numThreads, i) 72 | group.Add(int(i)) 73 | } 74 | 75 | /*Done should be called on threads routine exit to notify Wait that thread has 76 | * finished*/ 77 | func Done() { 78 | atomic.AddInt32(&numThreads, -1) 79 | group.Done() 80 | } 81 | 82 | /*Setup initializes shutdown framework. Setup signal listener for SIGINT, 83 | * SIGTERM */ 84 | func Setup() { 85 | atomic.StoreInt32(&flag, 0) 86 | atomic.StoreInt32(&numThreads, 0) 87 | sigs := make(chan os.Signal, 1) 88 | signal.Notify(sigs, syscall.SIGINT, syscall.SIGTERM) 89 | 90 | Context, cancel = context.WithCancel(context.Background()) 91 | 92 | Register(1) 93 | go func() { 94 | defer Done() 95 | select { 96 | case sig := <-sigs: 97 | if sig == syscall.SIGINT || sig == syscall.SIGTERM { 98 | Initiate() 99 | } 100 | case <-InitiatedCh(): 101 | } 102 | }() 103 | } 104 | 105 | /*Wait waits for all the registered threads to terminate*/ 106 | func Wait() { 107 | group.Wait() 108 | } 109 | 110 | /*NumProcs returns number of currently running threads*/ 111 | func NumProcs() int32 { 112 | return atomic.LoadInt32(&numThreads) 113 | } 114 | 115 | /*InitiateAndWait is a helper which is often used in tests, where we want to 116 | Initiate shutdown and Wait program to shutdown on function exit*/ 117 | func InitiateAndWait() { 118 | Initiate() 119 | Wait() 120 | } 121 | -------------------------------------------------------------------------------- /shutdown/shutdown_test.go: -------------------------------------------------------------------------------- 1 | // Copyright (c) 2017 Uber Technologies, Inc. 2 | // 3 | // Permission is hereby granted, free of charge, to any person obtaining a copy 4 | // of this software and associated documentation files (the "Software"), to deal 5 | // in the Software without restriction, including without limitation the rights 6 | // to use, copy, modify, merge, publish, distribute, sublicense, and/or sell 7 | // copies of the Software, and to permit persons to whom the Software is 8 | // furnished to do so, subject to the following conditions: 9 | // 10 | // The above copyright notice and this permission notice shall be included in 11 | // all copies or substantial portions of the Software. 12 | // 13 | // THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR 14 | // IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, 15 | // FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE 16 | // AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER 17 | // LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, 18 | // OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN 19 | // THE SOFTWARE. 20 | 21 | package shutdown 22 | 23 | import ( 24 | "fmt" 25 | "os" 26 | "syscall" 27 | "testing" 28 | "time" 29 | 30 | "github.com/uber/storagetapper/config" 31 | "github.com/uber/storagetapper/log" 32 | ) 33 | 34 | const numProcs int32 = 57 35 | 36 | func sigTerm(t *testing.T) { 37 | err := syscall.Kill(syscall.Getpid(), syscall.SIGTERM) 38 | if err != nil { 39 | fmt.Printf("TestBool: Error: %v", err) 40 | t.FailNow() 41 | } 42 | } 43 | 44 | func worker() { 45 | <-InitiatedCh() 46 | Done() 47 | } 48 | 49 | func TestBasic(t *testing.T) { 50 | log.Debugf("TestBasic") 51 | Setup() 52 | Register(numProcs) 53 | for i := int32(0); i < numProcs; i++ { 54 | go worker() 55 | } 56 | if Initiated() { 57 | t.Fatalf("Shouldn't be initiated") 58 | } 59 | Initiate() 60 | if !Initiated() { 61 | t.Fatalf("Should be initiated") 62 | } 63 | Wait() 64 | if NumProcs() != 0 { 65 | t.FailNow() 66 | } 67 | log.Debugf("TestBasicFinished") 68 | } 69 | 70 | func TestInitiateAndWait(t *testing.T) { 71 | log.Debugf("TestInitiateAndWait") 72 | Setup() 73 | Register(numProcs) 74 | for i := int32(0); i < numProcs; i++ { 75 | go worker() 76 | } 77 | if Initiated() { 78 | t.Fatalf("Shouldn't be initiated") 79 | } 80 | InitiateAndWait() 81 | if NumProcs() != 0 { 82 | t.FailNow() 83 | } 84 | log.Debugf("TestInitiateAndWaitFinished") 85 | } 86 | 87 | func TestSignal(t *testing.T) { 88 | log.Debugf("TestSignal") 89 | Setup() 90 | Register(numProcs) 91 | for i := int32(0); i < numProcs; i++ { 92 | go worker() 93 | } 94 | sigTerm(t) 95 | Wait() 96 | if NumProcs() != 0 { 97 | t.FailNow() 98 | } 99 | log.Debugf("TestSignalFinished") 100 | } 101 | 102 | func pollingWorker() { 103 | for !Initiated() { 104 | time.Sleep(time.Millisecond * 50) 105 | } 106 | Done() 107 | } 108 | 109 | func TestBool(t *testing.T) { 110 | log.Debugf("TestBool") 111 | Setup() 112 | Register(numProcs) 113 | for i := int32(0); i < numProcs; i++ { 114 | go pollingWorker() 115 | } 116 | sigTerm(t) 117 | Wait() 118 | if NumProcs() != 0 { 119 | t.FailNow() 120 | } 121 | log.Debugf("TestBoolFinished") 122 | } 123 | 124 | func TestMain(m *testing.M) { 125 | cfg := config.Get() 126 | log.Configure(cfg.LogType, cfg.LogLevel, false) 127 | os.Exit(m.Run()) 128 | } 129 | -------------------------------------------------------------------------------- /snapshot/snapshot_test.go: -------------------------------------------------------------------------------- 1 | // Copyright (c) 2017 Uber Technologies, Inc. 2 | // 3 | // Permission is hereby granted, free of charge, to any person obtaining a copy 4 | // of this software and associated documentation files (the "Software"), to deal 5 | // in the Software without restriction, including without limitation the rights 6 | // to use, copy, modify, merge, publish, distribute, sublicense, and/or sell 7 | // copies of the Software, and to permit persons to whom the Software is 8 | // furnished to do so, subject to the following conditions: 9 | // 10 | // The above copyright notice and this permission notice shall be included in 11 | // all copies or substantial portions of the Software. 12 | // 13 | // THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR 14 | // IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, 15 | // FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE 16 | // AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER 17 | // LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, 18 | // OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN 19 | // THE SOFTWARE. 20 | 21 | package snapshot 22 | 23 | import ( 24 | "os" 25 | "testing" 26 | 27 | "github.com/uber/storagetapper/log" 28 | "github.com/uber/storagetapper/shutdown" 29 | "github.com/uber/storagetapper/state" 30 | "github.com/uber/storagetapper/test" 31 | ) 32 | 33 | func TestCreateNonExistent(t *testing.T) { 34 | _, err := Start("not_existent_plugin", "", "", "", "", nil, nil, nil) 35 | test.Assert(t, err != nil, "should return error") 36 | } 37 | 38 | func TestMain(m *testing.M) { 39 | cfg = test.LoadConfig() 40 | 41 | shutdown.Setup() 42 | defer func() { 43 | shutdown.Initiate() 44 | shutdown.Wait() 45 | }() 46 | 47 | if err := state.InitManager(shutdown.Context, cfg); err != nil { 48 | log.Errorf("State init failed") 49 | os.Exit(1) 50 | } 51 | defer state.Close() 52 | 53 | if m.Run() != 0 { 54 | os.Exit(1) 55 | } 56 | 57 | runBenchmarks() 58 | } 59 | -------------------------------------------------------------------------------- /state/clusters.go: -------------------------------------------------------------------------------- 1 | // Copyright (c) 2017 Uber Technologies, Inc. 2 | // 3 | // Permission is hereby granted, free of charge, to any person obtaining a copy 4 | // of this software and associated documentation files (the "Software"), to deal 5 | // in the Software without restriction, including without limitation the rights 6 | // to use, copy, modify, merge, publish, distribute, sublicense, and/or sell 7 | // copies of the Software, and to permit persons to whom the Software is 8 | // furnished to do so, subject to the following conditions: 9 | // 10 | // The above copyright notice and this permission notice shall be included in 11 | // all copies or substantial portions of the Software. 12 | // 13 | // THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR 14 | // IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, 15 | // FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE 16 | // AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER 17 | // LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, 18 | // OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN 19 | // THE SOFTWARE. 20 | 21 | package state 22 | 23 | import ( 24 | "database/sql" 25 | 26 | "github.com/uber/storagetapper/db" 27 | "github.com/uber/storagetapper/log" 28 | "github.com/uber/storagetapper/util" 29 | ) 30 | 31 | //InsertClusterInfo adds connection information for the cluster "name" to state 32 | func InsertClusterInfo(ci *db.Addr) error { 33 | err := util.ExecSQL(mgr.conn, "INSERT INTO clusters(name,host,port,user,password) VALUES(?, ?, ?, ?, ?)", 34 | ci.Name, ci.Host, ci.Port, ci.User, ci.Pwd) 35 | if log.E(err) { 36 | return err 37 | } 38 | log.Debugf("Cluster added: name:%v Host:%v Port:%v User:%v", ci.Name, ci.Host, ci.Port, ci.User) 39 | return nil 40 | } 41 | 42 | //DeleteClusterInfo delete cluster connection info from state database 43 | func DeleteClusterInfo(name string) error { 44 | err := util.ExecSQL(mgr.conn, "DELETE FROM clusters WHERE name=?", name) 45 | if log.E(err) { 46 | return err 47 | } 48 | log.Debugf("Cluster deleted: %+v", name) 49 | return nil 50 | } 51 | 52 | //GetClusterInfo lists cluster connection info from state database 53 | func GetClusterInfo(cond string, args ...interface{}) ([]db.Addr, error) { 54 | log.Debugf("List clusters %v", cond) 55 | rows, err := util.QuerySQL(mgr.conn, "SELECT name,host,port,user,password FROM clusters "+cond, args...) 56 | if err != nil { 57 | return nil, err 58 | } 59 | defer func() { log.E(rows.Close()) }() 60 | res := make([]db.Addr, 0) 61 | var r db.Addr 62 | for rows.Next() { 63 | if err := rows.Scan(&r.Name, &r.Host, &r.Port, &r.User, &r.Pwd); err != nil { 64 | return nil, err 65 | } 66 | res = append(res, r) 67 | } 68 | if err := rows.Err(); err != nil { 69 | return nil, err 70 | } 71 | return res, nil 72 | } 73 | 74 | func selectCluster(a *db.Addr, connType db.ConnectionType) error { 75 | query := "SELECT name,host,port,user,password FROM clusters WHERE name=? AND type=?" 76 | return util.QueryRowSQL(mgr.conn, query, a.Name, connType.String()).Scan(&a.Name, &a.Host, &a.Port, &a.User, &a.Pwd) 77 | } 78 | 79 | //ConnectInfoGet resolves database address using state clusters table 80 | //If no Slaves we need to fall back to master 81 | func ConnectInfoGet(l *db.Loc, connType db.ConnectionType) (*db.Addr, error) { 82 | var a db.Addr 83 | 84 | //FIXME: Add connection type to info in state 85 | 86 | if mgr.conn == nil { 87 | log.Debugf("State hasn't been initialized yet") 88 | return nil, nil 89 | } 90 | 91 | a.Name = l.Cluster 92 | if l.Cluster == "" { 93 | err := util.QueryRowSQL(mgr.conn, "SELECT cluster FROM state WHERE service=? AND db=?", l.Service, 94 | l.Name).Scan(&a.Name) 95 | if err != nil && err != sql.ErrNoRows { 96 | log.E(err) 97 | return nil, err 98 | } 99 | if a.Name != "" { 100 | log.Debugf("Cluster name resolved from state: %v by service=%v, db=%v, connType=%v", a.Name, l.Service, 101 | l.Name, connType) 102 | } 103 | } 104 | 105 | err := selectCluster(&a, connType) 106 | //If the connection type is not available then check if the other type is 107 | if err == sql.ErrNoRows && connType == db.Slave { 108 | err = selectCluster(&a, db.Master) 109 | } else if err == sql.ErrNoRows && connType == db.Master { 110 | err = selectCluster(&a, db.Slave) 111 | } 112 | 113 | if err == sql.ErrNoRows { 114 | return nil, nil 115 | } 116 | 117 | if log.E(err) { 118 | return nil, err 119 | } 120 | 121 | if l.Cluster != "" && a.Name != l.Cluster { 122 | log.Errorf("Cluster name mismatch, given: %v, in state %v", l.Cluster, a.Name) 123 | return nil, nil 124 | } 125 | 126 | a.DB = l.Name 127 | 128 | return &a, nil 129 | } 130 | -------------------------------------------------------------------------------- /state/clusters_test.go: -------------------------------------------------------------------------------- 1 | // Copyright (c) 2017 Uber Technologies, Inc. 2 | // 3 | // Permission is hereby granted, free of charge, to any person obtaining a copy 4 | // of this software and associated documentation files (the "Software"), to deal 5 | // in the Software without restriction, including without limitation the rights 6 | // to use, copy, modify, merge, publish, distribute, sublicense, and/or sell 7 | // copies of the Software, and to permit persons to whom the Software is 8 | // furnished to do so, subject to the following conditions: 9 | // 10 | // The above copyright notice and this permission notice shall be included in 11 | // all copies or substantial portions of the Software. 12 | // 13 | // THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR 14 | // IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, 15 | // FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE 16 | // AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER 17 | // LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, 18 | // OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN 19 | // THE SOFTWARE. 20 | 21 | package state 22 | 23 | import ( 24 | "testing" 25 | 26 | "github.com/stretchr/testify/assert" 27 | "github.com/uber/storagetapper/db" 28 | "github.com/uber/storagetapper/test" 29 | ) 30 | 31 | //Structure for input of TestConnectInfoGet test 32 | type testConnectInfoGetInput struct { 33 | clusterName string 34 | connType db.ConnectionType 35 | } 36 | 37 | func TestConnectInfoGet(t *testing.T) { 38 | tests := []struct { 39 | name string 40 | input testConnectInfoGetInput 41 | output *db.Addr 42 | }{ 43 | { 44 | "Test row exists and master type", 45 | testConnectInfoGetInput{ 46 | "test_cluster_db1", 47 | db.Master, 48 | }, 49 | &db.Addr{ 50 | Name: "test_cluster_db1", 51 | Host: "MasterHost", 52 | Port: 123, 53 | User: "maximus", 54 | Pwd: "pass123", 55 | DB: "test_db", 56 | }, 57 | }, 58 | { 59 | "Test row exists and slave type", 60 | testConnectInfoGetInput{ 61 | "test_cluster_db1", 62 | db.Slave, 63 | }, 64 | &db.Addr{ 65 | Name: "test_cluster_db1", 66 | Host: "SlaveHost", 67 | Port: 123, 68 | User: "maximus", 69 | Pwd: "pass123", 70 | DB: "test_db", 71 | }, 72 | }, 73 | { 74 | "Test row exists and query for slave but only master type exists", 75 | testConnectInfoGetInput{ 76 | "test_cluster_db2", 77 | db.Slave, 78 | }, 79 | &db.Addr{ 80 | Name: "test_cluster_db2", 81 | Host: "MasterHost", 82 | Port: 123, 83 | User: "maximus", 84 | Pwd: "pass123", 85 | DB: "test_db", 86 | }, 87 | }, 88 | { 89 | "Test row exists and query for master but only slave type exists", 90 | testConnectInfoGetInput{ 91 | "test_cluster_db3", 92 | db.Master, 93 | }, 94 | &db.Addr{ 95 | Name: "test_cluster_db3", 96 | Host: "SlaveHost", 97 | Port: 123, 98 | User: "maximus", 99 | Pwd: "pass123", 100 | DB: "test_db", 101 | }, 102 | }, 103 | { 104 | "Test no row exists", 105 | testConnectInfoGetInput{ 106 | "test_cluster_db4", 107 | db.Master, 108 | }, 109 | nil, 110 | }, 111 | } 112 | 113 | resetState(t) 114 | 115 | //Schema(name, host, port, user, password, type) 116 | test.ExecSQL(mgr.conn, t, `INSERT INTO clusters(name, host, port, user, password, type) values(?,?,?,?,?,?)`, "test_cluster_db1", "MasterHost", 123, "maximus", "pass123", db.Master.String()) 117 | test.ExecSQL(mgr.conn, t, `INSERT INTO clusters(name, host, port, user, password, type) values(?,?,?,?,?,?)`, "test_cluster_db1", "SlaveHost", 123, "maximus", "pass123", db.Slave.String()) 118 | test.ExecSQL(mgr.conn, t, `INSERT INTO clusters(name, host, port, user, password, type) values(?,?,?,?,?,?)`, "test_cluster_db2", "MasterHost", 123, "maximus", "pass123", db.Master.String()) 119 | test.ExecSQL(mgr.conn, t, `INSERT INTO clusters(name, host, port, user, password, type) values(?,?,?,?,?,?)`, "test_cluster_db3", "SlaveHost", 123, "maximus", "pass123", db.Slave.String()) 120 | 121 | for _, tt := range tests { 122 | t.Run(tt.name, func(t *testing.T) { 123 | l := &db.Loc{Service: "test_svc1", Name: "test_db", Cluster: tt.input.clusterName} 124 | dbInfo, _ := ConnectInfoGet(l, tt.input.connType) 125 | if tt.output == nil { 126 | assert.Equal(t, dbInfo, tt.output) 127 | } else { 128 | assert.Equal(t, dbInfo.Host, tt.output.Host) 129 | assert.Equal(t, dbInfo.Name, tt.output.Name) 130 | } 131 | }) 132 | } 133 | } 134 | -------------------------------------------------------------------------------- /state/schema.go: -------------------------------------------------------------------------------- 1 | // Copyright (c) 2017 Uber Technologies, Inc. 2 | // 3 | // Permission is hereby granted, free of charge, to any person obtaining a copy 4 | // of this software and associated documentation files (the "Software"), to deal 5 | // in the Software without restriction, including without limitation the rights 6 | // to use, copy, modify, merge, publish, distribute, sublicense, and/or sell 7 | // copies of the Software, and to permit persons to whom the Software is 8 | // furnished to do so, subject to the following conditions: 9 | // 10 | // The above copyright notice and this permission notice shall be included in 11 | // all copies or substantial portions of the Software. 12 | // 13 | // THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR 14 | // IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, 15 | // FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE 16 | // AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER 17 | // LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, 18 | // OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN 19 | // THE SOFTWARE. 20 | 21 | package state 22 | 23 | import ( 24 | "encoding/json" 25 | 26 | "github.com/uber/storagetapper/log" 27 | "github.com/uber/storagetapper/types" 28 | "github.com/uber/storagetapper/util" 29 | ) 30 | 31 | //SchemaRow represents state schema row 32 | type SchemaRow struct { 33 | Name string `json:"name"` 34 | Type string `json:"type"` 35 | Body string `json:"body"` 36 | } 37 | 38 | //InsertSchema inserts output schema into state 39 | func InsertSchema(name string, typ string, schema string) error { 40 | err := util.ExecSQL(mgr.conn, "INSERT INTO output_schema(name,type,schema_body) VALUES(?, ?, ?)", name, typ, schema) 41 | if log.E(err) { 42 | return err 43 | } 44 | log.Debugf("Schema added: name:%v type: %v schema:%v", name, typ, schema) 45 | return nil 46 | } 47 | 48 | //UpdateSchema inserts output schema into state 49 | func UpdateSchema(name string, typ string, schema string) error { 50 | err := util.ExecSQL(mgr.conn, "UPDATE output_schema SET schema_body=? WHERE name=? AND type=?", schema, name, typ) 51 | if log.E(err) { 52 | return err 53 | } 54 | log.Debugf("Schema updated: name:%v type: %v schema:%v", name, typ, schema) 55 | return nil 56 | } 57 | 58 | //DeleteSchema deletes output schema from the state 59 | func DeleteSchema(name string, typ string) error { 60 | err := util.ExecSQL(mgr.conn, "DELETE FROM output_schema WHERE name=? AND type=?", name, typ) 61 | if log.E(err) { 62 | return err 63 | } 64 | log.Debugf("Schema deleted: %+v %+v", name, typ) 65 | return nil 66 | } 67 | 68 | //GetOutputSchema returns output schema from the state 69 | func GetOutputSchema(name string, typ string) string { 70 | var body string 71 | 72 | err := util.QueryRowSQL(mgr.conn, "SELECT schema_body FROM output_schema WHERE name=? AND type=?", name, typ).Scan(&body) 73 | 74 | if err != nil { 75 | if err.Error() != "sql: no rows in result set" { 76 | log.E(err) 77 | } 78 | return "" 79 | } 80 | log.Debugf("Return output schema from state, name: %v, type: %v body: %v", name, typ, body) 81 | return body 82 | } 83 | 84 | //SchemaGet is builtin schema resolver 85 | func SchemaGet(namespace string, schemaName string, typ string) (*types.AvroSchema, error) { 86 | var err error 87 | var a *types.AvroSchema 88 | 89 | s := GetOutputSchema(schemaName, typ) 90 | if s != "" { 91 | a = &types.AvroSchema{} 92 | err = json.Unmarshal([]byte(s), a) 93 | //TODO: Implement proper schema version handling in state 94 | if a.SchemaVersion == 0 { 95 | a.SchemaVersion = 1 96 | } 97 | } 98 | 99 | return a, err 100 | } 101 | 102 | //ListOutputSchema lists schemas from state, filtered by cond 103 | func ListOutputSchema(cond string, args ...interface{}) ([]SchemaRow, error) { 104 | log.Debugf("List schemas %v", cond) 105 | rows, err := util.QuerySQL(mgr.conn, "SELECT name,type,schema_body FROM output_schema "+cond, args...) 106 | if err != nil { 107 | return nil, err 108 | } 109 | defer func() { log.E(rows.Close()) }() 110 | res := make([]SchemaRow, 0) 111 | var r SchemaRow 112 | for rows.Next() { 113 | if err := rows.Scan(&r.Name, &r.Type, &r.Body); err != nil { 114 | return nil, err 115 | } 116 | res = append(res, r) 117 | } 118 | if err := rows.Err(); err != nil { 119 | return nil, err 120 | } 121 | return res, nil 122 | } 123 | -------------------------------------------------------------------------------- /test/env.go: -------------------------------------------------------------------------------- 1 | // Copyright (c) 2017 Uber Technologies, Inc. 2 | // 3 | // Permission is hereby granted, free of charge, to any person obtaining a copy 4 | // of this software and associated documentation files (the "Software"), to deal 5 | // in the Software without restriction, including without limitation the rights 6 | // to use, copy, modify, merge, publish, distribute, sublicense, and/or sell 7 | // copies of the Software, and to permit persons to whom the Software is 8 | // furnished to do so, subject to the following conditions: 9 | // 10 | // The above copyright notice and this permission notice shall be included in 11 | // all copies or substantial portions of the Software. 12 | // 13 | // THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR 14 | // IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, 15 | // FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE 16 | // AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER 17 | // LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, 18 | // OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN 19 | // THE SOFTWARE. 20 | 21 | package test 22 | 23 | import ( 24 | "database/sql" 25 | "fmt" 26 | "os" 27 | "path" 28 | "runtime" 29 | "strings" 30 | "testing" 31 | "time" 32 | 33 | "github.com/Shopify/sarama" 34 | 35 | "github.com/uber/storagetapper/config" 36 | "github.com/uber/storagetapper/db" 37 | "github.com/uber/storagetapper/log" 38 | "github.com/uber/storagetapper/metrics" 39 | "github.com/uber/storagetapper/types" 40 | "github.com/uber/storagetapper/util" 41 | ) 42 | 43 | var cfg *config.AppConfig 44 | 45 | //Failer introduced to allow both testing.T and testing.B in CheckFail call 46 | type Failer interface { 47 | FailNow() 48 | Skip(...interface{}) 49 | } 50 | 51 | // ExecSQL executes SQL and logs on error. 52 | func ExecSQL(db *sql.DB, t Failer, query string, param ...interface{}) { 53 | CheckFail(util.ExecSQL(db, query, param...), t) 54 | } 55 | 56 | // CheckFail fails the test if error is set, logs file, line, func of the failure 57 | // location 58 | func CheckFail(err error, t Failer) { 59 | if err != nil { 60 | pc, file, no, _ := runtime.Caller(1) 61 | details := runtime.FuncForPC(pc) 62 | log.Fatalf("%v:%v %v: Test failed: %v", path.Base(file), no, path.Base(details.Name()), err.Error()) 63 | t.FailNow() 64 | } 65 | } 66 | 67 | //MySQLAvailable test if local MySQL instance is running 68 | func MySQLAvailable() bool { 69 | d, err := db.Open(&db.Addr{Host: "localhost", Port: 3306, User: types.TestMySQLUser, Pwd: types.TestMySQLPassword}) 70 | if err != nil { 71 | return false 72 | } 73 | if err := d.Close(); err != nil { 74 | return false 75 | } 76 | return true 77 | } 78 | 79 | func kafkaAvailable() bool { 80 | producer, err := sarama.NewSyncProducer(cfg.Pipe.Kafka.Addresses, nil) 81 | if err != nil { 82 | return false 83 | } 84 | _ = producer.Close() 85 | return true 86 | } 87 | 88 | //SkipIfNoKafkaAvailable tries to connect to local Kafka and if fails, then skip 89 | //the test 90 | func SkipIfNoKafkaAvailable(t Failer) { 91 | if !kafkaAvailable() { 92 | t.Skip("No local Kafka detected") 93 | } 94 | } 95 | 96 | //SkipIfNoMySQLAvailable tries to connect to local MySQL and if fails, then skip 97 | //the test 98 | func SkipIfNoMySQLAvailable(t Failer) { 99 | if !MySQLAvailable() { 100 | t.Skip("No local MySQL detected") 101 | } 102 | } 103 | 104 | // Assert fails the test if cond is false, logs file, line, func of the failure 105 | // location 106 | func Assert(t *testing.T, cond bool, param ...interface{}) { 107 | if !cond { 108 | pc, file, no, _ := runtime.Caller(1) 109 | details := runtime.FuncForPC(pc) 110 | args := []interface{}{path.Base(file), no, path.Base(details.Name())} 111 | msg := "Assertion failed" 112 | if len(param) > 0 { 113 | var ok bool 114 | if msg, ok = param[0].(string); !ok { 115 | msg = strings.Repeat("%v ", len(param)) 116 | } 117 | args = append(args, param[1:]...) 118 | } 119 | _, _ = fmt.Fprintf(os.Stderr, time.Now().Format(time.RFC3339Nano)+" ASSERT %v:%v %v: "+msg+"\n", args...) 120 | t.FailNow() 121 | } 122 | } 123 | 124 | // LoadConfig loads config for testing environment 125 | func LoadConfig() *config.AppConfig { 126 | cfg = config.Get() 127 | 128 | log.Configure(cfg.LogType, cfg.LogLevel, config.Environment() == config.EnvProduction) 129 | 130 | err := metrics.Init() 131 | log.F(err) 132 | 133 | db.GetConnInfo = db.GetConnInfoForTest 134 | db.GetEnumerator = db.GetEnumeratorForTest 135 | db.IsValidConn = db.IsValidConnForTest 136 | 137 | log.Debugf("Config: %+v", cfg) 138 | 139 | return cfg 140 | } 141 | -------------------------------------------------------------------------------- /test/env_test.go: -------------------------------------------------------------------------------- 1 | // Copyright (c) 2017 Uber Technologies, Inc. 2 | // 3 | // Permission is hereby granted, free of charge, to any person obtaining a copy 4 | // of this software and associated documentation files (the "Software"), to deal 5 | // in the Software without restriction, including without limitation the rights 6 | // to use, copy, modify, merge, publish, distribute, sublicense, and/or sell 7 | // copies of the Software, and to permit persons to whom the Software is 8 | // furnished to do so, subject to the following conditions: 9 | // 10 | // The above copyright notice and this permission notice shall be included in 11 | // all copies or substantial portions of the Software. 12 | // 13 | // THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR 14 | // IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, 15 | // FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE 16 | // AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER 17 | // LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, 18 | // OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN 19 | // THE SOFTWARE. 20 | 21 | package test 22 | 23 | import ( 24 | "testing" 25 | 26 | "github.com/uber/storagetapper/db" 27 | "github.com/uber/storagetapper/types" 28 | ) 29 | 30 | func TestBasic(t *testing.T) { 31 | _ = LoadConfig() 32 | 33 | SkipIfNoMySQLAvailable(t) 34 | 35 | CheckFail(nil, t) 36 | 37 | conn, err := db.Open(&db.Addr{Host: "localhost", Port: 3306, User: types.TestMySQLUser, Pwd: types.TestMySQLPassword}) 38 | CheckFail(err, t) 39 | 40 | ExecSQL(conn, t, "SELECT 1") 41 | ExecSQL(conn, t, "SELECT 1+?", 52) 42 | } 43 | -------------------------------------------------------------------------------- /test/wait.go: -------------------------------------------------------------------------------- 1 | // Copyright (c) 2017 Uber Technologies, Inc. 2 | // 3 | // Permission is hereby granted, free of charge, to any person obtaining a copy 4 | // of this software and associated documentation files (the "Software"), to deal 5 | // in the Software without restriction, including without limitation the rights 6 | // to use, copy, modify, merge, publish, distribute, sublicense, and/or sell 7 | // copies of the Software, and to permit persons to whom the Software is 8 | // furnished to do so, subject to the following conditions: 9 | // 10 | // The above copyright notice and this permission notice shall be included in 11 | // all copies or substantial portions of the Software. 12 | // 13 | // THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR 14 | // IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, 15 | // FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE 16 | // AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER 17 | // LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, 18 | // OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN 19 | // THE SOFTWARE. 20 | 21 | package test 22 | 23 | import ( 24 | "time" 25 | 26 | "github.com/uber/storagetapper/shutdown" 27 | ) 28 | 29 | // WaitForNumProc waits when number of running procs registered in shutdown 30 | //framework becomes less or equal to the given number 31 | //Returns false if the timeout expires and number of procs still above the limit 32 | func WaitForNumProc(n int32, timeout time.Duration) bool { 33 | for shutdown.NumProcs() > n && timeout > 0 { 34 | time.Sleep(200 * time.Millisecond) 35 | timeout -= 200 * time.Millisecond 36 | } 37 | 38 | return shutdown.NumProcs() <= n 39 | } 40 | 41 | // WaitForNumProcGreater opposite to WaitForNumProc 42 | func WaitForNumProcGreater(n int32, timeout time.Duration) bool { 43 | for shutdown.NumProcs() < n && timeout > 0 { 44 | time.Sleep(200 * time.Millisecond) 45 | timeout -= 200 * time.Millisecond 46 | } 47 | 48 | return shutdown.NumProcs() >= n 49 | } 50 | -------------------------------------------------------------------------------- /test/wait_test.go: -------------------------------------------------------------------------------- 1 | // Copyright (c) 2017 Uber Technologies, Inc. 2 | // 3 | // Permission is hereby granted, free of charge, to any person obtaining a copy 4 | // of this software and associated documentation files (the "Software"), to deal 5 | // in the Software without restriction, including without limitation the rights 6 | // to use, copy, modify, merge, publish, distribute, sublicense, and/or sell 7 | // copies of the Software, and to permit persons to whom the Software is 8 | // furnished to do so, subject to the following conditions: 9 | // 10 | // The above copyright notice and this permission notice shall be included in 11 | // all copies or substantial portions of the Software. 12 | // 13 | // THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR 14 | // IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, 15 | // FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE 16 | // AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER 17 | // LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, 18 | // OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN 19 | // THE SOFTWARE. 20 | 21 | package test 22 | 23 | import ( 24 | "testing" 25 | "time" 26 | 27 | "github.com/uber/storagetapper/shutdown" 28 | ) 29 | 30 | func TestWaitForNumProc(t *testing.T) { 31 | if !WaitForNumProc(1, 200*time.Millisecond) { 32 | t.Fatalf("There should be 0 threads") 33 | } 34 | 35 | ch := make(chan bool) 36 | a := func() { 37 | defer shutdown.Done() 38 | <-ch 39 | } 40 | 41 | shutdown.Register(2) 42 | go a() 43 | go a() 44 | 45 | if WaitForNumProc(1, 200*time.Millisecond) { 46 | t.Fatalf("Should timeout because there is 2 threads") 47 | } 48 | 49 | ch <- true 50 | 51 | if !WaitForNumProc(1, 200*time.Millisecond) { 52 | t.Fatalf("Should succeed so as we signaled one proc to finish %v", shutdown.NumProcs()) 53 | } 54 | 55 | ch <- true 56 | 57 | if !WaitForNumProc(0, 200*time.Millisecond) { 58 | t.Fatalf("Should succeed so as we signaled second proc to finish") 59 | } 60 | } 61 | 62 | func TestWaitForNumProcGreater(t *testing.T) { 63 | if !WaitForNumProcGreater(0, 200*time.Millisecond) { 64 | t.Fatalf("There should be 0 threads") 65 | } 66 | 67 | ch := make(chan bool) 68 | a := func() { 69 | defer shutdown.Done() 70 | <-ch 71 | } 72 | 73 | shutdown.Register(2) 74 | go a() 75 | go a() 76 | 77 | if WaitForNumProcGreater(3, 200*time.Millisecond) { 78 | t.Fatalf("Should timeout because there is 2 threads") 79 | } 80 | 81 | ch <- true 82 | 83 | if !WaitForNumProcGreater(1, 200*time.Millisecond) { 84 | t.Fatalf("Should succeed so as we signaled one proc to finish %v", shutdown.NumProcs()) 85 | } 86 | 87 | ch <- true 88 | 89 | if !WaitForNumProcGreater(0, 200*time.Millisecond) { 90 | t.Fatalf("Should succeed so as we signaled second proc to finish") 91 | } 92 | } 93 | -------------------------------------------------------------------------------- /throttle/throttle.go: -------------------------------------------------------------------------------- 1 | package throttle 2 | 3 | import "time" 4 | 5 | //Throttle implements generic Throttler 6 | type Throttle struct { 7 | CheckInterval int64 8 | NumSamples int64 9 | Target int64 10 | 11 | current int64 12 | 13 | samples []int64 14 | sumSamples int64 15 | samplesHand int64 16 | 17 | ticker *time.Ticker 18 | } 19 | 20 | //New creates new Throttler 21 | func New(target int64, checkInterval int64, numSamples int64) *Throttle { 22 | return &Throttle{checkInterval, numSamples, target, 0, make([]int64, numSamples), 0, 0, time.NewTicker(time.Microsecond * time.Duration(checkInterval))} 23 | } 24 | 25 | func (t *Throttle) tick() int64 { 26 | t.sumSamples -= t.samples[t.samplesHand] 27 | t.sumSamples += t.current 28 | t.samples[t.samplesHand] = t.current 29 | t.current = 0 30 | t.samplesHand++ 31 | if t.samplesHand >= t.NumSamples { 32 | t.samplesHand = 0 33 | } 34 | 35 | avg := t.sumSamples / t.NumSamples 36 | if avg <= t.Target { 37 | return 0 38 | } 39 | 40 | return t.CheckInterval - t.CheckInterval*t.Target/(avg) 41 | } 42 | 43 | //Advice return a sleep advice to satisfy throttling requirements 44 | func (t *Throttle) Advice(add int64) int64 { 45 | var s int64 46 | 47 | t.current += add 48 | 49 | if t.Target == 0 { 50 | return 0 51 | } 52 | 53 | select { 54 | case <-t.ticker.C: 55 | s = t.tick() 56 | default: 57 | } 58 | 59 | return s 60 | } 61 | 62 | //Close releases resources 63 | func (t *Throttle) Close() { 64 | t.ticker.Stop() 65 | } 66 | -------------------------------------------------------------------------------- /throttle/throttle_test.go: -------------------------------------------------------------------------------- 1 | package throttle 2 | 3 | import ( 4 | "os" 5 | "testing" 6 | 7 | "github.com/uber/storagetapper/config" 8 | "github.com/uber/storagetapper/test" 9 | ) 10 | 11 | var cfg *config.AppConfig 12 | 13 | /* 14 | func testStep(n int64, target int64, iops int64, interval int64, samples int64, t *testing.T) { 15 | log.Debugf("n=%v target=%v iops=%v interval=%v samples=%v", n, target, iops, interval, samples) 16 | 17 | h := New(target, interval, samples) 18 | 19 | d := n * interval / target 20 | 21 | time.Sleep(time.Microsecond * time.Duration(interval)) 22 | 23 | for i := 0; i < int(samples); i++ { 24 | h.Advice(iops) 25 | time.Sleep(time.Microsecond * time.Duration(interval)) 26 | } 27 | 28 | var sum int64 29 | //for i := 0; i < 10; i++ { 30 | for n > 0 { 31 | s := h.Advice(iops) 32 | if s < 0 || s > interval { 33 | t.Fatalf("Throttle time is out of bounds: %v", s) 34 | } 35 | sum += s 36 | time.Sleep(time.Microsecond * time.Duration(interval)) 37 | n -= target 38 | } 39 | 40 | log.Debugf("Total duration: %v, throttled time: %v", d, sum) 41 | log.Debugf("Got: %v, Expected: %v", float64(sum)/float64(d), float64(iops-target)/float64(iops)) 42 | 43 | if target > iops { 44 | if sum != 0 { 45 | t.Fatalf("No throttle should be applied, got %v, maximum %v", sum, d) 46 | } 47 | } else if iops*(n/target) < n { 48 | if sum != d { 49 | t.Fatalf("Maximum throttle should be applied, got %v, maximum %v", sum, d) 50 | } 51 | } else if int64((float64(sum)*100/float64(d)))-int64((float64(iops-target)*100/float64(iops))) != 0 && int64((float64(sum)*1000/float64(d)+5)/100)-int64((float64(iops-target)*1000/float64(iops)+5)/100) != 0 { 52 | t.Fatalf("Got: %v, Expected: %v", int64((float64(sum)*1000/float64(d)+5)/100), int64((float64(iops-target)*1000/float64(iops)+5)/100)) 53 | } 54 | } 55 | 56 | func TestThrottleBasic(t *testing.T) { 57 | t.Skip("Flaky") 58 | for i := int64(1); i <= 128; i *= 2 { 59 | for j := int64(1); j <= 128; j *= 2 { 60 | testStep(1024*i, i, j, 200, 5, t) 61 | } 62 | } 63 | } 64 | 65 | func TestZeroThrottleBasic(t *testing.T) { 66 | t.Skip("Flaky") 67 | for j := int64(1); j <= 128; j *= 2 { 68 | h := New(0, 200, 5) 69 | s := h.Advice(j) 70 | if s != 0 { 71 | t.Fatalf("No throttling should happen for zero target") 72 | } 73 | } 74 | } 75 | */ 76 | 77 | func TestMain(m *testing.M) { 78 | cfg = test.LoadConfig() 79 | os.Exit(m.Run()) 80 | } 81 | -------------------------------------------------------------------------------- /types/avro.go: -------------------------------------------------------------------------------- 1 | // Copyright (c) 2017 Uber Technologies, Inc. 2 | // 3 | // Permission is hereby granted, free of charge, to any person obtaining a copy 4 | // of this software and associated documentation files (the "Software"), to deal 5 | // in the Software without restriction, including without limitation the rights 6 | // to use, copy, modify, merge, publish, distribute, sublicense, and/or sell 7 | // copies of the Software, and to permit persons to whom the Software is 8 | // furnished to do so, subject to the following conditions: 9 | // 10 | // The above copyright notice and this permission notice shall be included in 11 | // all copies or substantial portions of the Software. 12 | // 13 | // THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR 14 | // IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, 15 | // FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE 16 | // AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER 17 | // LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, 18 | // OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN 19 | // THE SOFTWARE. 20 | 21 | package types 22 | 23 | //AvroPrimitiveType is declared to improve readability AvroSchema/AvroFields 24 | //declarations 25 | type AvroPrimitiveType string 26 | 27 | //AvroComplexType is declared to improve readability AvroSchema/AvroFields 28 | type AvroComplexType string 29 | 30 | //Avro data types 31 | const ( 32 | AvroNULL AvroPrimitiveType = "null" 33 | AvroBOOLEAN AvroPrimitiveType = "boolean" 34 | AvroINT AvroPrimitiveType = "int" 35 | AvroLONG AvroPrimitiveType = "long" 36 | AvroFLOAT AvroPrimitiveType = "float" 37 | AvroDOUBLE AvroPrimitiveType = "double" 38 | AvroBYTES AvroPrimitiveType = "bytes" 39 | AvroSTRING AvroPrimitiveType = "string" 40 | 41 | AvroRECORD AvroComplexType = "record" 42 | ) 43 | 44 | // AvroSchema represents the structure of Avro schema format 45 | type AvroSchema struct { 46 | Fields []AvroField `json:"fields"` 47 | Name string `json:"name"` 48 | Namespace string `json:"namespace"` 49 | Owner string `json:"owner"` 50 | SchemaVersion int `json:",omitempty"` 51 | Type AvroComplexType `json:"type"` 52 | Doc string `json:"doc,omitempty"` 53 | LastModified string `json:"last_modified,omitempty"` 54 | } 55 | 56 | // AvroField represents structure of each of the fields in the schema 57 | type AvroField struct { 58 | Name string `json:"name"` 59 | Type []AvroPrimitiveType `json:"type"` 60 | Default interface{} `json:"default,omitempty"` 61 | Doc string `json:"doc,omitempty"` 62 | } 63 | -------------------------------------------------------------------------------- /types/const.go: -------------------------------------------------------------------------------- 1 | // Copyright (c) 2017 Uber Technologies, Inc. 2 | // 3 | // Permission is hereby granted, free of charge, to any person obtaining a copy 4 | // of this software and associated documentation files (the "Software"), to deal 5 | // in the Software without restriction, including without limitation the rights 6 | // to use, copy, modify, merge, publish, distribute, sublicense, and/or sell 7 | // copies of the Software, and to permit persons to whom the Software is 8 | // furnished to do so, subject to the following conditions: 9 | // 10 | // The above copyright notice and this permission notice shall be included in 11 | // all copies or substantial portions of the Software. 12 | // 13 | // THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR 14 | // IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, 15 | // FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE 16 | // AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER 17 | // LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, 18 | // OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN 19 | // THE SOFTWARE. 20 | 21 | package types 22 | 23 | //MyDBName is a database name this service will use to save state to 24 | var MyDBName = "storagetapper" 25 | 26 | //MySvcName is a name of this service 27 | const MySvcName = "storagetapper" 28 | 29 | //TestMySQLUser is MySQL user used to connect to MySQL in test 30 | var TestMySQLUser = "storagetapper" 31 | 32 | //TestMySQLPassword is MySQL password used to connect to MySQL in test 33 | var TestMySQLPassword = "storagetapper" 34 | 35 | //MySQLBoolean represents MySQL equivalent of boolean type 36 | const MySQLBoolean = "tinyint(1)" 37 | 38 | //MyClusterName is a cluster name this service will use to save state to 39 | var MyClusterName = "storagetapper" 40 | -------------------------------------------------------------------------------- /types/format.go: -------------------------------------------------------------------------------- 1 | // Copyright (c) 2017 Uber Technologies, Inc. 2 | // 3 | // Permission is hereby granted, free of charge, to any person obtaining a copy 4 | // of this software and associated documentation files (the "Software"), to deal 5 | // in the Software without restriction, including without limitation the rights 6 | // to use, copy, modify, merge, publish, distribute, sublicense, and/or sell 7 | // copies of the Software, and to permit persons to whom the Software is 8 | // furnished to do so, subject to the following conditions: 9 | // 10 | // The above copyright notice and this permission notice shall be included in 11 | // all copies or substantial portions of the Software. 12 | // 13 | // THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR 14 | // IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, 15 | // FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE 16 | // AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER 17 | // LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, 18 | // OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN 19 | // THE SOFTWARE. 20 | 21 | package types 22 | 23 | //go:generate msgp 24 | 25 | //CommonFormatEvent types 26 | const ( 27 | Insert int = iota 28 | Delete int = iota 29 | Schema int = iota 30 | ) 31 | 32 | //CommonFormatField represents single field of the CommonFormatEvent 33 | type CommonFormatField struct { 34 | Name string 35 | Value interface{} 36 | } 37 | 38 | //CommonFormatEvent is a generic format which represents single data 39 | //modification event 40 | type CommonFormatEvent struct { 41 | Type string //insert, delete, schema 42 | Key []interface{} 43 | SeqNo uint64 44 | Timestamp int64 //This only used for metrics, to measure time in buffer 45 | Fields *[]CommonFormatField `json:",omitempty"` 46 | } 47 | -------------------------------------------------------------------------------- /types/input.go: -------------------------------------------------------------------------------- 1 | // Copyright (c) 2017 Uber Technologies, Inc. 2 | // 3 | // Permission is hereby granted, free of charge, to any person obtaining a copy 4 | // of this software and associated documentation files (the "Software"), to deal 5 | // in the Software without restriction, including without limitation the rights 6 | // to use, copy, modify, merge, publish, distribute, sublicense, and/or sell 7 | // copies of the Software, and to permit persons to whom the Software is 8 | // furnished to do so, subject to the following conditions: 9 | // 10 | // The above copyright notice and this permission notice shall be included in 11 | // all copies or substantial portions of the Software. 12 | // 13 | // THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR 14 | // IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, 15 | // FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE 16 | // AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER 17 | // LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, 18 | // OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN 19 | // THE SOFTWARE. 20 | 21 | package types 22 | 23 | const ( 24 | // InputMySQL defines input of type to be MySQL 25 | InputMySQL = "mysql" 26 | ) 27 | -------------------------------------------------------------------------------- /types/schema.go: -------------------------------------------------------------------------------- 1 | // Copyright (c) 2017 Uber Technologies, Inc. 2 | // 3 | // Permission is hereby granted, free of charge, to any person obtaining a copy 4 | // of this software and associated documentation files (the "Software"), to deal 5 | // in the Software without restriction, including without limitation the rights 6 | // to use, copy, modify, merge, publish, distribute, sublicense, and/or sell 7 | // copies of the Software, and to permit persons to whom the Software is 8 | // furnished to do so, subject to the following conditions: 9 | // 10 | // The above copyright notice and this permission notice shall be included in 11 | // all copies or substantial portions of the Software. 12 | // 13 | // THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR 14 | // IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, 15 | // FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE 16 | // AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER 17 | // LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, 18 | // OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN 19 | // THE SOFTWARE. 20 | 21 | package types 22 | 23 | import ( 24 | "database/sql" 25 | ) 26 | 27 | // TableSchema represents table schema from information_schema 28 | type TableSchema struct { 29 | DBName string 30 | TableName string 31 | Columns []ColumnSchema 32 | } 33 | 34 | // ColumnSchema is struct for each Column schema 35 | type ColumnSchema struct { 36 | Name string 37 | OrdinalPosition uint64 38 | IsNullable string 39 | DataType string 40 | CharacterMaximumLength sql.NullInt64 41 | NumericPrecision sql.NullInt64 42 | NumericScale sql.NullInt64 43 | Type string 44 | Key string 45 | } 46 | -------------------------------------------------------------------------------- /types/types.go: -------------------------------------------------------------------------------- 1 | // Copyright (c) 2017 Uber Technologies, Inc. 2 | // 3 | // Permission is hereby granted, free of charge, to any person obtaining a copy 4 | // of this software and associated documentation files (the "Software"), to deal 5 | // in the Software without restriction, including without limitation the rights 6 | // to use, copy, modify, merge, publish, distribute, sublicense, and/or sell 7 | // copies of the Software, and to permit persons to whom the Software is 8 | // furnished to do so, subject to the following conditions: 9 | // 10 | // The above copyright notice and this permission notice shall be included in 11 | // all copies or substantial portions of the Software. 12 | // 13 | // THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR 14 | // IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, 15 | // FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE 16 | // AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER 17 | // LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, 18 | // OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN 19 | // THE SOFTWARE. 20 | 21 | package types 22 | 23 | import "time" 24 | 25 | /*RowMessage is used to pass message to the local streamer */ 26 | type RowMessage struct { 27 | Type int 28 | Key string 29 | Data *[]interface{} 30 | SeqNo uint64 31 | Timestamp time.Time 32 | } 33 | 34 | /*TableLoc - table location */ 35 | type TableLoc struct { 36 | Service string 37 | Cluster string 38 | DB string 39 | Table string 40 | Input string 41 | Output string 42 | Version int 43 | } 44 | -------------------------------------------------------------------------------- /util/mysql.go: -------------------------------------------------------------------------------- 1 | // Copyright (c) 2017 Uber Technologies, Inc. 2 | // 3 | // Permission is hereby granted, free of charge, to any person obtaining a copy 4 | // of this software and associated documentation files (the "Software"), to deal 5 | // in the Software without restriction, including without limitation the rights 6 | // to use, copy, modify, merge, publish, distribute, sublicense, and/or sell 7 | // copies of the Software, and to permit persons to whom the Software is 8 | // furnished to do so, subject to the following conditions: 9 | // 10 | // The above copyright notice and this permission notice shall be included in 11 | // all copies or substantial portions of the Software. 12 | // 13 | // THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR 14 | // IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, 15 | // FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE 16 | // AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER 17 | // LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, 18 | // OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN 19 | // THE SOFTWARE. 20 | 21 | package util 22 | 23 | //https://dev.mysql.com/doc/refman/8.0/en/string-literals.html 24 | func escapeBackslash(s string) string { 25 | j := 0 26 | buf := make([]byte, len(s)*2) 27 | 28 | for i := 0; i < len(s); i++ { 29 | switch s[i] { 30 | case '\x00': //00 31 | buf[j] = '\\' 32 | buf[j+1] = '0' 33 | j += 2 34 | case '\b': //08 35 | buf[j] = '\\' 36 | buf[j+1] = 'b' 37 | j += 2 38 | case '\n': //10 39 | buf[j] = '\\' 40 | buf[j+1] = 'n' 41 | j += 2 42 | case '\r': //13 43 | buf[j] = '\\' 44 | buf[j+1] = 'r' 45 | j += 2 46 | case '\x1a': //26 47 | buf[j] = '\\' 48 | buf[j+1] = 'Z' 49 | j += 2 50 | case '"': //34 51 | buf[j] = '\\' 52 | buf[j+1] = '"' 53 | j += 2 54 | case '\'': //39 55 | buf[j] = '\\' 56 | buf[j+1] = '\'' 57 | j += 2 58 | case '\\': //92 59 | buf[j] = '\\' 60 | buf[j+1] = '\\' 61 | j += 2 62 | default: 63 | buf[j] = s[i] 64 | j++ 65 | } 66 | } 67 | 68 | return string(buf[:j]) 69 | } 70 | 71 | // EscapeQuotes escapes given quote by doubling it 72 | func EscapeQuotes(s string, quote byte) string { 73 | j := 0 74 | buf := make([]byte, len(s)*2) 75 | 76 | for i := 0; i < len(s); i++ { 77 | buf[j] = s[i] 78 | j++ 79 | if s[i] == quote { 80 | buf[j] = quote 81 | j++ 82 | } 83 | } 84 | 85 | return string(buf[:j]) 86 | } 87 | 88 | // MySQLEscape returns escaped string 89 | // escaping backslashes or duplicating single quotes, based on backslash 90 | // parameter 91 | func MySQLEscape(backslash bool, s string) string { 92 | if backslash { 93 | return escapeBackslash(s) 94 | } 95 | return EscapeQuotes(s, '\'') 96 | } 97 | -------------------------------------------------------------------------------- /util/mysql_test.go: -------------------------------------------------------------------------------- 1 | // Copyright (c) 2017 Uber Technologies, Inc. 2 | // 3 | // Permission is hereby granted, free of charge, to any person obtaining a copy 4 | // of this software and associated documentation files (the "Software"), to deal 5 | // in the Software without restriction, including without limitation the rights 6 | // to use, copy, modify, merge, publish, distribute, sublicense, and/or sell 7 | // copies of the Software, and to permit persons to whom the Software is 8 | // furnished to do so, subject to the following conditions: 9 | // 10 | // The above copyright notice and this permission notice shall be included in 11 | // all copies or substantial portions of the Software. 12 | // 13 | // THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR 14 | // IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, 15 | // FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE 16 | // AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER 17 | // LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, 18 | // OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN 19 | // THE SOFTWARE. 20 | 21 | package util 22 | 23 | import ( 24 | "testing" 25 | 26 | "github.com/stretchr/testify/require" 27 | ) 28 | 29 | var testsANSIQuotes = []struct { 30 | name string 31 | src string 32 | res string 33 | quote byte 34 | }{ 35 | {"simple_single", "a", "a", '\''}, 36 | {"simple_double", "a", "a", '"'}, 37 | {"empty_single", "", "", '\''}, 38 | {"empty_double", "", "", '"'}, 39 | {"double_in_single", `"`, `"`, '\''}, 40 | {"single_in_double", "'", "'", '"'}, 41 | {"2double_in_single", `""`, `""`, '\''}, 42 | {"2single_in_double", "''", "''", '"'}, 43 | {"single_in_single", "'", "''", '\''}, 44 | {"double_in_double", `"`, `""`, '"'}, 45 | {"2single_in_single", "''", "''''", '\''}, 46 | {"2double_in_double", `""`, `""""`, '"'}, 47 | {"txt_single_in_single", "a'a'", "a''a''", '\''}, 48 | {"txt_double_in_double", `a"a"`, `a""a""`, '"'}, 49 | {"all_escapes_single", "\x00\b\n\r\x1a\"'\\", "\x00\b\n\r\x1a\"''\\", '\''}, 50 | {"all_escapes_double", "\x00\b\n\r\x1a\"'\\", "\x00\b\n\r\x1a\"\"'\\", '"'}, 51 | } 52 | 53 | var testsMySQLEscape = []struct { 54 | name string 55 | src string 56 | res string 57 | }{ 58 | {"simple", "a", "a"}, 59 | {"all_escapes", "\x00\b\n\r\x1a\"'\\", `\0\b\n\r\Z\"\'\\`}, 60 | {"all_escapes_doubled", "\x00\x00\b\b\n\n\r\r\x1a\x1a\"\"''\\\\", `\0\0\b\b\n\n\r\r\Z\Z\"\"\'\'\\\\`}, 61 | {"txt_all_escapes", "a\x00a\ba\na\ra\x1aa\"a'a\\a", `a\0a\ba\na\ra\Za\"a\'a\\a`}, 62 | {"txt_all_escapes_doubled", "a\x00\x00a\b\ba\n\na\r\ra\x1a\x1aa\"\"a''a\\\\a", `a\0\0a\b\ba\n\na\r\ra\Z\Za\"\"a\'\'a\\\\a`}, 63 | } 64 | 65 | func TestANSIQuotes(t *testing.T) { 66 | for _, v := range testsANSIQuotes { 67 | t.Run(v.name, func(t *testing.T) { 68 | res := EscapeQuotes(v.src, v.quote) 69 | require.Equal(t, v.res, res) 70 | }) 71 | } 72 | } 73 | 74 | func TestMySQLEscape(t *testing.T) { 75 | for _, v := range testsMySQLEscape { 76 | t.Run(v.name, func(t *testing.T) { 77 | res := MySQLEscape(true, v.src) 78 | require.Equal(t, v.res, res) 79 | }) 80 | } 81 | } 82 | 83 | func TestMySQLEscapeNoBackslash(t *testing.T) { 84 | for i := 0; i < len(testsANSIQuotes); i += 2 { 85 | v := testsANSIQuotes[i] 86 | t.Run(v.name, func(t *testing.T) { 87 | res := MySQLEscape(false, v.src) 88 | require.Equal(t, v.res, res) 89 | }) 90 | } 91 | } 92 | --------------------------------------------------------------------------------