├── .gitignore ├── .travis.yml ├── LICENSE ├── README.md ├── project.clj ├── recordbus.example.conf ├── resources └── log4j.properties └── src ├── log4j.properties └── org └── spootnik ├── recordbus.clj └── recordbus ├── config.clj ├── kafka.clj └── sql.clj /.gitignore: -------------------------------------------------------------------------------- 1 | /target 2 | /classes 3 | /checkouts 4 | pom.xml 5 | pom.xml.asc 6 | *.jar 7 | *.class 8 | /.lein-* 9 | /.nrepl-port 10 | .hgignore 11 | .hg/ 12 | -------------------------------------------------------------------------------- /.travis.yml: -------------------------------------------------------------------------------- 1 | sudo: false 2 | language: clojure 3 | lein: lein2 4 | jdk: 5 | - openjdk7 6 | - oraclejdk7 7 | branches: 8 | except: 9 | - gh-pages 10 | -------------------------------------------------------------------------------- /LICENSE: -------------------------------------------------------------------------------- 1 | Copyright 2015 Pierre-Yves Ritschard 2 | 3 | Permission to use, copy, modify, and distribute this software for any 4 | purpose with or without fee is hereby granted, provided that the above 5 | notice and this permission notice appear in all copies. 6 | 7 | THE SOFTWARE IS PROVIDED "AS IS" AND THE AUTHOR DISCLAIMS ALL WARRANTIES 8 | WITH REGARD TO THIS SOFTWARE INCLUDING ALL IMPLIED WARRANTIES OF 9 | MERCHANTABILITY AND FITNESS. IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR 10 | ANY SPECIAL, DIRECT, INDIRECT, OR CONSEQUENTIAL DAMAGES OR ANY DAMAGES 11 | WHATSOEVER RESULTING FROM LOSS OF USE, DATA OR PROFITS, WHETHER IN AN 12 | ACTION OF CONTRACT, NEGLIGENCE OR OTHER TORTIOUS ACTION, ARISING OUT OF 13 | OR IN CONNECTION WITH THE USE OR PERFORMANCE OF THIS SOFTWARE. 14 | -------------------------------------------------------------------------------- /README.md: -------------------------------------------------------------------------------- 1 | recordbus: stream SQL replication events to Apache Kafka. 2 | =========================================================== 3 | 4 | 5 | [![Build Status](https://secure.travis-ci.org/pyr/recordbus.png)](http://travis-ci.org/pyr/recordbus) 6 | 7 | 8 | Recordbus connects to a MySQL instance as a replicant and 9 | produces the replication events read onto an Apache Kafka 10 | topic. The events are produced as a JSON-serialized map, keyed 11 | by the server-id which produced the event. 12 | 13 | ## Configuration 14 | 15 | Recordbus accepts a single argument, its configuration file 16 | which has the following format: 17 | 18 | ``` 19 | mysql.host=localhost 20 | mysql.port=3306 21 | mysql.user=replicant 22 | mysql.password=replicant 23 | topic=recordbus 24 | bootstrap.servers=localhost:9092 25 | ``` 26 | 27 | ## Building and running 28 | 29 | You'll need [leiningen](http://leiningen.org) to build the project, you can then 30 | run `lein uberjar` to build the project. 31 | 32 | To run it, just issue `java -jar target/recordbus-0.1.0-standalone.jar ` 33 | 34 | Pre-built JARs are available in https://github.com/pyr/recordbus/releases/ 35 | 36 | ## Use-cases 37 | 38 | - Live cache updates from MySQL 39 | - Materialized views from MySQL events 40 | 41 | ## Event Types 42 | 43 | Each JSON payload in the stream contains a `type` key which 44 | determines the rest of the payload's shape. 45 | 46 | These three fields will be present in all events: 47 | 48 | - `type`: one of `unknown`, `start-v3`, `query`, `stop`, `rotate`, `intvar`, 49 | `load`, `slave`, `create-file`, `append-block`, `exec-load`, `delete-file`, `new-load`, 50 | `user-var`, `format-description`, `xid`, `begin-load-query`, `execute-load-query`, 51 | `table-map`, `pre-ga-write-rows`, `pre-ga-update-rows`, `pre-ga-delete-rows`, 52 | `write-rows`, `update-rows`, `delete-rows`, `incident`, `heartbeat`, `ignorable`, 53 | `rows-query`, `ext-write-rows`, `ext-update-rows`, `ext-delete-rows`, `gtid`, 54 | `anonymous-gtid`, `previous-gtids`. 55 | - `timestamp`: the timestamp of the event 56 | - `server-id`: server-id from which this request originated, also used as the record key. 57 | 58 | Here are the key event-type-specific fields: 59 | 60 | 61 | 62 | 63 | 64 | 65 | 66 | 67 | 68 | 69 | 70 | 71 | 72 |
typefields
format-descriptionbinlog-version, server-version, header-length
gtidgtid, flags
querysql, error-code, database, exec-time
rotatebinlog-filename, binlog-position
rows-querybinlog-filename
table-mapdatabase, table, column-types, column-metadata, column-nullability
update-rowscols-old, cols-new, rows, table-id
write-rowscols, rows, table-id
delete-rowscols, rows, table-id
xidxid
73 | 74 | 75 | 76 | Assuming a client had the following conversation with 77 | a MySQL server: 78 | 79 | ```SQL 80 | create database foobar; 81 | use foobar; 82 | create table user ( 83 | id int primary key not null auto_increment, 84 | login varchar(255), 85 | name varchar(255) 86 | ); 87 | insert into user(login,name) values('bob','bob'); 88 | insert into user(login,name) values('bill','bill'); 89 | update user set name="Billy Bob" where id=2; 90 | delete from user where id=1; 91 | ``` 92 | 93 | The following JSON payloads would be produced in the kafka topic: 94 | 95 | #### Preamble 96 | 97 | ```json 98 | {"server-id":1, 99 | "timestamp":0, 100 | "binlog-position":574, 101 | "binlog-filename":"mysql-bin.000064", 102 | "type":"rotate"} 103 | {"server-id":1, 104 | "timestamp":1426511022000, 105 | "header-length":19, 106 | "server-version":"10.0.17-MariaDB-log", 107 | "binlog-version":4, 108 | "type":"format-description"} 109 | ``` 110 | 111 | #### Create database and Create table 112 | 113 | ```json 114 | {"server-id":1, 115 | "timestamp":1426512246000, 116 | "exec-time":0, 117 | "database":"", 118 | "error-code":0, 119 | "sql":"# Dum", 120 | "type":"query"} 121 | {"server-id":1, 122 | "timestamp":1426512246000, 123 | "exec-time":0, 124 | "database":"foobar", 125 | "error-code":0, 126 | "sql":"create database foobar", 127 | "type":"query"} 128 | {"server-id":1, 129 | "timestamp":1426512336000, 130 | "exec-time":0, 131 | "database":"", 132 | "error-code":0, 133 | "sql":"# Dum", 134 | "type":"query"} 135 | {"server-id":1, 136 | "timestamp":1426512336000, 137 | "exec-time":0, 138 | "database":"foobar", 139 | "error-code":0, 140 | "sql":"create table user ( id int primary key not null auto_increment, login varchar(255), name varchar(255))", 141 | "type":"query"} 142 | ``` 143 | 144 | #### Inserts 145 | 146 | ```json 147 | {"server-id":1, 148 | "timestamp":1426512368000, 149 | "exec-time":0, 150 | "database":"", 151 | "error-code":0, 152 | "sql":"BEGIN", 153 | "type":"query"} 154 | {"server-id":1, 155 | "timestamp":1426512368000, 156 | "column-nullability":[1,2], 157 | "column-metadata":[0,765,765], 158 | "column-types":[3,15,15], 159 | "table":"user", 160 | "database":"foobar", 161 | "type":"table-map"} 162 | {"server-id":1, 163 | "timestamp":1426512368000, 164 | "table-id":73, 165 | "rows":[["1","bob","bob"]], 166 | "cols":[0,1,2], 167 | "type":"write-rows"} 168 | {"server-id":1, 169 | "timestamp":1426512368000, 170 | "xid":32, 171 | "type":"xid"} 172 | 173 | {"server-id":1, 174 | "timestamp":1426512383000, 175 | "exec-time":0, 176 | "database":"", 177 | "error-code":0, 178 | "sql":"BEGIN", 179 | "type":"query"} 180 | {"server-id":1, 181 | "timestamp":1426512383000, 182 | "column-nullability":[1,2], 183 | "column-metadata":[0,765,765], 184 | "column-types":[3,15,15], 185 | "table":"user", 186 | "database":"foobar", 187 | "type":"table-map"} 188 | {"server-id":1, 189 | "timestamp":1426512383000, 190 | "table-id":73, 191 | "rows":[["2","bill","bill"]], 192 | "cols":[0,1,2], 193 | "type":"write-rows"} 194 | {"server-id":1, 195 | "timestamp":1426512383000, 196 | "xid":33, 197 | "type":"xid"} 198 | ``` 199 | 200 | #### Updates 201 | 202 | ```json 203 | {"server-id":1, 204 | "timestamp":1426512399000, 205 | "exec-time":0, 206 | "database":"", 207 | "error-code":0, 208 | "sql":"BEGIN", 209 | "type":"query"} 210 | {"server-id":1, 211 | "timestamp":1426512399000, 212 | "column-nullability":[1,2], 213 | "column-metadata":[0,765,765], 214 | "column-types":[3,15,15], 215 | "table":"user", 216 | "database":"foobar", 217 | "type":"table-map"} 218 | {"server-id":1, 219 | "timestamp":1426512399000, 220 | "table-id":73, 221 | "rows":[[["2","bill","bill"],["2","bill","Billy Bob"]]], 222 | "cols-new":[0,1,2], 223 | "cols-old":[0,1,2], 224 | "type":"update-rows"} 225 | {"server-id":1, 226 | "timestamp":1426512399000, 227 | "xid":34, 228 | "type":"xid"} 229 | ``` 230 | 231 | #### Deletes 232 | 233 | ```json 234 | {"server-id":1, 235 | "timestamp":1426512411000, 236 | "exec-time":0, 237 | "database":"", 238 | "error-code":0, 239 | "sql":"BEGIN", 240 | "type":"query"} 241 | {"server-id":1, 242 | "timestamp":1426512411000, 243 | "column-nullability":[1,2], 244 | "column-metadata":[0,765,765], 245 | "column-types":[3,15,15], 246 | "table":"user", 247 | "database":"foobar", 248 | "type":"table-map"} 249 | {"server-id":1, 250 | "timestamp":1426512411000, 251 | "table-id":73, 252 | "rows":[["1","bob","bob"]], 253 | "cols":[0,1,2], 254 | "type":"delete-rows"} 255 | {"server-id":1, 256 | "timestamp":1426512411000, 257 | "xid":35, 258 | "type":"xid"} 259 | ``` 260 | 261 | ## Caveats 262 | 263 | - No support for keeping track of offsets 264 | - No configurable key or serialization 265 | - MySQL only (a PostgreSQL implementation would be nice) 266 | 267 | ## License 268 | 269 | Copyright 2015 Pierre-Yves Ritschard 270 | 271 | Permission to use, copy, modify, and distribute this software for any 272 | purpose with or without fee is hereby granted, provided that the above 273 | notice and this permission notice appear in all copies. 274 | 275 | THE SOFTWARE IS PROVIDED "AS IS" AND THE AUTHOR DISCLAIMS ALL WARRANTIES 276 | WITH REGARD TO THIS SOFTWARE INCLUDING ALL IMPLIED WARRANTIES OF 277 | MERCHANTABILITY AND FITNESS. IN NO EVENT SHALL THE AUTHOR BE LIABLE FOR 278 | ANY SPECIAL, DIRECT, INDIRECT, OR CONSEQUENTIAL DAMAGES OR ANY DAMAGES 279 | WHATSOEVER RESULTING FROM LOSS OF USE, DATA OR PROFITS, WHETHER IN AN 280 | ACTION OF CONTRACT, NEGLIGENCE OR OTHER TORTIOUS ACTION, ARISING OUT OF 281 | OR IN CONNECTION WITH THE USE OR PERFORMANCE OF THIS SOFTWARE. 282 | -------------------------------------------------------------------------------- /project.clj: -------------------------------------------------------------------------------- 1 | (defproject org.spootnik/recordbus "0.1.1" 2 | :description "MySQL binlog to kafka" 3 | :url "https://github.com/pyr/recordbus" 4 | :license {:name "Eclipse Public License" 5 | :url "http://www.eclipse.org/legal/epl-v10.html"} 6 | :aot :all 7 | :main org.spootnik.recordbus 8 | :dependencies [[org.clojure/clojure "1.7.0"] 9 | [org.clojure/tools.logging "0.3.1"] 10 | [spootnik/unilog "0.7.8"] 11 | [cheshire "5.5.0"] 12 | [com.github.shyiko/mysql-binlog-connector-java "0.2.0"] 13 | [org.apache.kafka/kafka-clients "0.8.2.1"]]) 14 | -------------------------------------------------------------------------------- /recordbus.example.conf: -------------------------------------------------------------------------------- 1 | mysql.host=localhost 2 | mysql.port=3306 3 | mysql.user=replicant 4 | mysql.password=replicant 5 | topic=sqlstream 6 | bootstrap.servers=localhost:9092 7 | -------------------------------------------------------------------------------- /resources/log4j.properties: -------------------------------------------------------------------------------- 1 | log4j.rootLogger=DEBUG, A1 2 | log4j.appender.A1=org.apache.log4j.ConsoleAppender 3 | log4j.appender.A1.layout=org.apache.log4j.PatternLayout 4 | log4j.appender.A1.layout.ConversionPattern=%d %-5p %c: %m%n 5 | log4j.logger.org.apache.kafka=WARN 6 | -------------------------------------------------------------------------------- /src/log4j.properties: -------------------------------------------------------------------------------- 1 | log4j.rootLogger=INFO, A1 2 | log4j.appender.A1=org.apache.log4j.ConsoleAppender 3 | log4j.appender.A1.layout=org.apache.log4j.PatternLayout 4 | log4j.appender.A1.layout.ConversionPattern=%d %-5p %c: %m%n 5 | log4j.logger.org.apache.zookeeper=WARN 6 | log4j.logger.org.apache.kafka=WARN 7 | -------------------------------------------------------------------------------- /src/org/spootnik/recordbus.clj: -------------------------------------------------------------------------------- 1 | (ns org.spootnik.recordbus 2 | (:gen-class) 3 | (:require [org.spootnik.recordbus.sql :as sql] 4 | [org.spootnik.recordbus.kafka :as kafka] 5 | [org.spootnik.recordbus.config :as config] 6 | [cheshire.core :as json] 7 | [clojure.tools.logging :as log])) 8 | 9 | (defn callback 10 | [{:keys [kafka topic]}] 11 | (let [producer (kafka/producer kafka)] 12 | (fn [{:keys [server-id] :as event}] 13 | (kafka/publish 14 | producer 15 | (kafka/record topic server-id (json/generate-string event)))))) 16 | 17 | (defn -main 18 | [& [path]] 19 | (let [cfg (config/read-props (or path "/etc/recordbus.conf")) 20 | client (sql/replication-client (:sql cfg) (callback cfg))] 21 | (log/info "connecting to mysql") 22 | (loop [] 23 | (try 24 | (sql/connect! client) 25 | (catch java.io.IOException e)) 26 | (log/info "no replication connection open, retry in 500ms") 27 | (Thread/sleep 500) 28 | (recur)))) 29 | -------------------------------------------------------------------------------- /src/org/spootnik/recordbus/config.clj: -------------------------------------------------------------------------------- 1 | (ns org.spootnik.recordbus.config 2 | (:import java.util.Properties 3 | java.io.FileInputStream)) 4 | 5 | (defn read-props 6 | [path] 7 | (let [props (doto (Properties.) (.load (FileInputStream. path))) 8 | mysql {:host (.remove props "mysql.host") 9 | :port (Long/parseLong (.remove props "mysql.port")) 10 | :user (.remove props "mysql.user") 11 | :password (.remove props "mysql.password")} 12 | topic (or (.remove props "topic") "recordbus")] 13 | {:sql mysql :kafka props :topic topic})) 14 | -------------------------------------------------------------------------------- /src/org/spootnik/recordbus/kafka.clj: -------------------------------------------------------------------------------- 1 | (ns org.spootnik.recordbus.kafka 2 | "Very thin wrapper around kafka producer code" 3 | (:import org.apache.kafka.clients.producer.Producer 4 | org.apache.kafka.clients.producer.KafkaProducer 5 | org.apache.kafka.clients.producer.ProducerRecord 6 | java.nio.ByteBuffer)) 7 | 8 | (def key-serializer 9 | "org.apache.kafka.common.serialization.ByteArraySerializer") 10 | 11 | (def value-serializer 12 | "org.apache.kafka.common.serialization.StringSerializer") 13 | 14 | (defn producer 15 | [^java.util.Properties props] 16 | (.setProperty props "key.serializer" key-serializer) 17 | (.setProperty props "value.serializer" value-serializer) 18 | (KafkaProducer. props)) 19 | 20 | (defn record 21 | [^String topic ^Long key ^String payload] 22 | (let [key-data (-> (ByteBuffer/allocate 8) (.putLong key) (.array))] 23 | (ProducerRecord. topic key-data payload))) 24 | 25 | (defn publish 26 | [^KafkaProducer producer ^ProducerRecord record] 27 | (.send producer record)) 28 | -------------------------------------------------------------------------------- /src/org/spootnik/recordbus/sql.clj: -------------------------------------------------------------------------------- 1 | (ns org.spootnik.recordbus.sql 2 | "Publish SQL replication events on a queue." 3 | (:import com.github.shyiko.mysql.binlog.BinaryLogClient 4 | com.github.shyiko.mysql.binlog.BinaryLogClient$EventListener 5 | com.github.shyiko.mysql.binlog.BinaryLogClient$LifecycleListener 6 | com.github.shyiko.mysql.binlog.event.Event 7 | com.github.shyiko.mysql.binlog.event.EventType)) 8 | 9 | (def event-types 10 | {EventType/UNKNOWN :unknown 11 | EventType/START_V3 :start-v3 12 | EventType/QUERY :query 13 | EventType/STOP :stop 14 | EventType/ROTATE :rotate 15 | EventType/INTVAR :intvar 16 | EventType/LOAD :load 17 | EventType/SLAVE :slave 18 | EventType/CREATE_FILE :create-file 19 | EventType/APPEND_BLOCK :append-block 20 | EventType/EXEC_LOAD :exec-load 21 | EventType/DELETE_FILE :delete-file 22 | EventType/NEW_LOAD :new-load 23 | EventType/RAND :rand 24 | EventType/USER_VAR :user-var 25 | EventType/FORMAT_DESCRIPTION :format-description 26 | EventType/XID :xid 27 | EventType/BEGIN_LOAD_QUERY :begin-load-query 28 | EventType/EXECUTE_LOAD_QUERY :execute-load-query 29 | EventType/TABLE_MAP :table-map 30 | EventType/PRE_GA_WRITE_ROWS :pre-ga-write-rows 31 | EventType/PRE_GA_UPDATE_ROWS :pre-ga-update-rows 32 | EventType/PRE_GA_DELETE_ROWS :pre-ga-delete-rows 33 | EventType/WRITE_ROWS :write-rows 34 | EventType/UPDATE_ROWS :update-rows 35 | EventType/DELETE_ROWS :delete-rows 36 | EventType/INCIDENT :incident 37 | EventType/HEARTBEAT :heartbeat 38 | EventType/IGNORABLE :ignorable 39 | EventType/ROWS_QUERY :rows-query 40 | EventType/EXT_WRITE_ROWS :ext-write-rows 41 | EventType/EXT_UPDATE_ROWS :ext-update-rows 42 | EventType/EXT_DELETE_ROWS :ext-delete-rows 43 | EventType/GTID :gtid 44 | EventType/ANONYMOUS_GTID :anonymous-gtid 45 | EventType/PREVIOUS_GTIDS :previous-gtids}) 46 | 47 | (defn bitset-vec 48 | [^java.util.BitSet s] 49 | (loop [i 0 50 | res nil] 51 | (let [next (.nextSetBit s i)] 52 | (if (neg? next) 53 | (vec (reverse res)) 54 | (recur (inc next) (conj res next)))))) 55 | 56 | (defmulti augment-event-map :type) 57 | 58 | (defmethod augment-event-map :format-description 59 | [{:keys [data] :as event}] 60 | (assoc event 61 | :binlog-version (.getBinlogVersion data) 62 | :server-version (.getServerVersion data) 63 | :header-length (.getHeaderLength data))) 64 | 65 | (defmethod augment-event-map :gtid 66 | [{:keys [data] :as event}] 67 | (assoc event 68 | :gtid (.getGtid data) 69 | :flags (.getFlags data))) 70 | 71 | (defmethod augment-event-map :query 72 | [{:keys [data] :as event}] 73 | (assoc event 74 | :sql (.getSql data) 75 | :error-code (.getErrorCode data) 76 | :database (.getDatabase data) 77 | :exec-time (.getExecutionTime data))) 78 | 79 | (defmethod augment-event-map :rotate 80 | [{:keys [data] :as event}] 81 | (assoc event 82 | :binlog-filename (.getBinlogFilename data) 83 | :binlog-position (.getBinlogPosition data))) 84 | 85 | (defmethod augment-event-map :rows-query 86 | [{:keys [data] :as event}] 87 | (assoc event 88 | :binlog-filename (.getQuery data))) 89 | 90 | (defmethod augment-event-map :table-map 91 | [{:keys [data] :as event}] 92 | (assoc event 93 | :database (.getDatabase data) 94 | :table (.getTable data) 95 | :column-types (seq (.getColumnTypes data)) 96 | :column-metadata (seq (.getColumnMetadata data)) 97 | :column-nullability (bitset-vec (.getColumnNullability data)))) 98 | 99 | (defmethod augment-event-map :update-rows 100 | [{:keys [data] :as event}] 101 | (assoc event 102 | :cols-old (bitset-vec (.getIncludedColumnsBeforeUpdate data)) 103 | :cols-new (bitset-vec (.getIncludedColumns data)) 104 | :rows (for [[k v] (.getRows data)] [(mapv str k) (mapv str v)]) 105 | :table-id (.getTableId data))) 106 | 107 | (defmethod augment-event-map :write-rows 108 | [{:keys [data] :as event}] 109 | (assoc event 110 | :cols (bitset-vec (.getIncludedColumns data)) 111 | :rows (mapv (partial mapv str) (.getRows data)) 112 | :table-id (.getTableId data))) 113 | 114 | (defmethod augment-event-map :delete-rows 115 | [{:keys [data] :as event}] 116 | (assoc event 117 | :cols (bitset-vec (.getIncludedColumns data)) 118 | :rows (mapv (partial mapv str) (.getRows data)) 119 | :table-id (.getTableId data))) 120 | 121 | (defmethod augment-event-map :xid 122 | [{:keys [data] :as event}] 123 | (assoc event 124 | :xid (.getXid data))) 125 | 126 | (defmethod augment-event-map :default 127 | [event] 128 | event) 129 | 130 | (defn event->map 131 | [e] 132 | (let [header (.getHeader e) 133 | data (.getData e) 134 | type (-> (.getEventType header) event-types)] 135 | (assoc (augment-event-map {:type type :data data}) 136 | :timestamp (.getTimestamp header) 137 | :server-id (.getServerId header)))) 138 | 139 | (defn event-listener 140 | [callback] 141 | (reify 142 | BinaryLogClient$EventListener 143 | (onEvent [this payload] 144 | (callback (dissoc (event->map payload) :data))))) 145 | 146 | (defn replication-client 147 | [{:keys [host port user password]} callback] 148 | (doto (BinaryLogClient. host (int port) nil user password) 149 | (.setServerId 255) 150 | (.registerEventListener (event-listener callback)))) 151 | 152 | (defn connect! 153 | [client] 154 | (.connect client)) 155 | --------------------------------------------------------------------------------