def init_logging(): """ init logging config :param debug: :return: """ base_logger = logging.getLogger("synch") debug = Settings.debug() if debug: base_logger.setLevel(logging.DEBUG) else: base_logger.setLevel(logging.INFO) fmt = logging.Formatter( fmt="%(asctime)s - %(name)s:%(lineno)d - %(levelname)s - %(message)s", datefmt="%Y-%m-%d %H:%M:%S", ) sh = logging.StreamHandler(sys.stdout) sh.setLevel(logging.DEBUG) sh.setFormatter(fmt) base_logger.addHandler(sh) mail = Settings.get("mail") if mail: rate_limit = RateLimitingFilter(per=60) sh = logging.handlers.SMTPHandler( mailhost=mail.get("mailhost"), fromaddr=mail.get("fromaddr"), toaddrs=mail.get("toaddrs"), subject=mail.get("subject"), credentials=(mail.get("user"), mail.get("password")), ) sh.setLevel(logging.ERROR) sh.setFormatter(fmt) sh.addFilter(rate_limit) base_logger.addHandler(sh)
def get_writer(engine: ClickHouseEngine = None, choice=True) -> Union[ClickHouse, List[ClickHouse]]: """ get writer once """ writers = _writers.get(engine) if not choice: return writers if not writers: settings = Settings.get("clickhouse") hosts = settings.get("hosts") if Settings.is_cluster() and len(hosts) <= 1: raise ConfigurationError("hosts must more than one when cluster") for host in hosts: args = [host, settings.get("user"), settings.get("password"), Settings.cluster_name()] if engine == ClickHouseEngine.merge_tree.value: w = ClickHouseMergeTree(*args) elif engine == ClickHouseEngine.collapsing_merge_tree: w = ClickHouseCollapsingMergeTree(*args) elif engine == ClickHouseEngine.versioned_collapsing_merge_tree: w = ClickHouseVersionedCollapsingMergeTree(*args) elif engine == ClickHouseEngine.replacing_merge_tree or engine is None: w = ClickHouseReplacingMergeTree(*args) else: w = ClickHouse(*args) _writers.setdefault(engine, []).append(w) return random.choice(_writers.get(engine)) # nosec:B311
def __init__(self, alias): super().__init__(alias) source_db = Settings.get_source_db(alias) params = dict( host=source_db.get("host"), port=source_db.get("port"), user=source_db.get("user"), password=source_db.get("password"), ) self.insert_interval = Settings.insert_interval() self.skip_dmls = source_db.get("skip_dmls") or [] self.skip_update_tables = source_db.get("skip_update_tables") or [] self.skip_delete_tables = source_db.get("skip_delete_tables") or [] self.conn = psycopg2.connect(**params, cursor_factory=DictCursor) self.conn.set_isolation_level(ISOLATION_LEVEL_AUTOCOMMIT) self.cursor = self.conn.cursor() for database in source_db.get("databases"): database_name = database.get("database") replication_conn = psycopg2.connect( **params, database=database_name, connection_factory=LogicalReplicationConnection) self._repl_conn[database_name] = { "cursor": replication_conn.cursor(), }
def __init__(self, alias): super().__init__(alias) self.servers = Settings.get("kafka").get("servers") self.topic = f'{Settings.get("kafka").get("topic_prefix")}.{alias}' self.databases = Settings.get_source_db(alias).get("databases") self.producer = KafkaProducer( bootstrap_servers=self.servers, value_serializer=lambda x: json.dumps(x, cls=JsonEncoder).encode(), key_serializer=lambda x: x.encode(), ) self._init_topic()
def consume(ctx: Context, schema: str, skip_error: bool, last_msg_id: str): alias = ctx.obj["alias"] reader = get_reader(alias) tables = Settings.get_source_db_database_tables_name(alias, schema) tables_pk = {} for table in tables: tables_pk[table] = reader.get_primary_key(schema, table) # try etl full etl_full(alias, schema, tables_pk) table_dict = Settings.get_source_db_database_tables_dict(alias, schema) continuous_etl( alias, schema, tables_pk, table_dict, last_msg_id, skip_error, )
def __init__( self, alias: str, ): super().__init__() self.server_id = Settings.get_source_db(alias).get("server_id") self.pos_key = f"{self.prefix}:binlog:{alias}:{self.server_id}"
def __init__(self): """ init setting and create redis instance """ settings = Settings.get("redis") self.prefix = settings.get("prefix") self.queue_max_len = settings.get("queue_max_len") self.sentinel = settings.get("sentinel") if self.sentinel: sentinel = Sentinel(sentinels=map(lambda x: x.split(":"), settings.get("sentinel_hosts"))) kwargs = dict( service_name=settings.get("sentinel_master"), password=settings.get("password"), decode_responses=True, ) self.master = sentinel.master_for(**kwargs) self.slave = sentinel.slave_for(**kwargs) else: pool = redis.ConnectionPool( host=settings.get("host"), port=settings.get("port"), db=settings.get("db"), password=settings.get("password"), decode_responses=True, ) self.master = self.slave = redis.StrictRedis(connection_pool=pool)
def etl_full( alias: str, schema: str, tables_pk: Dict, renew=False, ): """ full etl """ reader = get_reader(alias) source_db_database = Settings.get_source_db_database(alias, schema) schema = source_db_database.get("database") writer = get_writer() if not writer.check_database_exists(schema): if source_db_database.get("auto_create") is not False: writer.create_database(schema) else: logger.warning( f"Can't etl since no database {schema} found in ClickHouse and auto_create=false" ) exit(-1) for table in source_db_database.get("tables"): if table.get("auto_full_etl") is False: continue table_name = table.get("table") pk = tables_pk.get(table_name) writer = get_writer(table.get("clickhouse_engine")) if not pk: logger.warning(f"No pk found in {schema}.{table_name}, skip") continue elif isinstance(pk, tuple): pk = f"({','.join(pk)})" if renew: drop_sql = f"drop table if exists {schema}.{table_name}" writer.execute(drop_sql) logger.info(f"drop table success:{schema}.{table_name}") if not writer.check_table_exists(schema, table_name): sign_column = table.get("sign_column") version_column = table.get("version_column") writer.execute( writer.get_table_create_sql( reader, schema, table_name, pk, table.get("partition_by"), table.get("engine_settings"), sign_column=sign_column, version_column=version_column, )) if reader.fix_column_type and not table.get("skip_decimal"): writer.fix_table_column_type(reader, schema, table_name) full_insert_sql = writer.get_full_insert_sql( reader, schema, table_name, sign_column) writer.execute(full_insert_sql) logger.info(f"full data etl for {schema}.{table_name} success") else: logger.debug( f"{schema}.{table_name} exists, skip, or use --renew force etl with drop old tables" )
def init(config_file): """ init """ Settings.init(config_file) init_logging() dsn = Settings.get("sentry", "dsn") if dsn: import sentry_sdk from sentry_sdk.integrations.redis import RedisIntegration sentry_sdk.init( dsn, environment=Settings.get("sentry", "environment"), integrations=[RedisIntegration()], ) if Settings.monitoring(): init_monitor_db()
def etl(ctx: Context, schema: str, renew: bool, table: List[str]): alias = ctx.obj["alias"] tables = table if not tables: tables = Settings.get_source_db_database_tables_name(alias, schema) tables_pk = {} reader = get_reader(alias) for table in tables: tables_pk[table] = reader.get_primary_key(schema, table) etl_full(alias, schema, tables_pk, renew)
def __init__(self, alias: str): self.alias = alias source_db = Settings.get_source_db(alias) self.source_db = source_db self.host = source_db.get("host") self.port = source_db.get("port") self.user = source_db.get("user") self.password = source_db.get("password") signal.signal(signal.SIGINT, self.signal_handler) signal.signal(signal.SIGTERM, self.signal_handler)
def get_broker(alias: str) -> Broker: b = _brokers.get(alias) broker_type = Settings.get_source_db(alias).get("broker_type") if not b: if broker_type == BrokerType.redis: b = RedisBroker(alias) elif broker_type == BrokerType.kafka: b = KafkaBroker(alias) else: raise NotImplementedError(f"Unsupported broker_type {broker_type}") _brokers[alias] = b return b
def check(ctx: Context, schema: str): alias = ctx.obj["alias"] reader = get_reader(alias) writer = get_writer() tables = Settings.get_source_db_database_tables_name(alias, schema) for table in tables: source_table_count = reader.get_count(schema, table) target_table_count = writer.get_count(schema, table) if source_table_count == target_table_count: logger.info(f"{schema}.{table} is equal, count={source_table_count}") else: logger.warning( f"{schema}.{table} is not equal, source_table_count={source_table_count}, target_table_count={target_table_count}" )
def insert_log( alias: str, schema: str, table: str, num: int, type_: int, ): if not Settings.monitoring(): return from synch.factory import get_writer now = datetime.datetime.now().strftime("%Y-%m-%d %H:%M:%S") sql = f"""INSERT INTO synch.log (alias, schema, table, num, type, created_at) VALUES ('{alias}', '{schema}', '{table}', {num}, {type_}, '{now}')""" get_writer().execute(sql)
def after_send(self, schema, table): now = int(time.time()) schema_table = f"{schema}.{table}" self.count.setdefault(schema_table, 0) self.count[schema_table] += 1 if self.last_time == 0: self.last_time = now if now - self.last_time >= Settings.insert_interval(): for schema_table, num in self.count.items(): logger.info(f"success send {num} events for {schema_table}") s, t = schema_table.split(".") insert_log(self.alias, s, t, num, 1) self.last_time = 0 self.count = {}
def get_writer(engine: ClickHouseEngine = None) -> ClickHouse: """ get writer once """ w = _writers.get(engine) if not w: settings = Settings.get("clickhouse") if engine == ClickHouseEngine.merge_tree.value: w = ClickHouseMergeTree(settings) elif engine == ClickHouseEngine.collapsing_merge_tree: w = ClickHouseCollapsingMergeTree(settings) elif engine == ClickHouseEngine.versioned_collapsing_merge_tree: w = ClickHouseVersionedCollapsingMergeTree(settings) elif engine == ClickHouseEngine.replacing_merge_tree or engine is None: w = ClickHouseReplacingMergeTree(settings) _writers[engine] = w return w
def init(cls, config): cls.settings = Settings.parse(config) broker_type = cls.settings.broker_type if broker_type == BrokerType.redis.value: cls.broker = RedisBroker(cls.settings) elif broker_type == BrokerType.kafka.value: cls.broker = KafkaBroker(cls.settings) cls.writer = ClickHouseWriter( host=cls.settings.clickhouse_host, port=cls.settings.clickhouse_port, password=cls.settings.clickhouse_password, user=cls.settings.clickhouse_user, ) if cls.settings.source_db == SourceDatabase.mysql.value: cls.reader = Mysql(cls.settings) elif cls.settings.source_db == SourceDatabase.postgres.value: cls.reader = Postgres(cls.settings)
def get_reader(alias: str) -> Reader: """ get reader once """ r = _readers.get(alias) if not r: source_db = Settings.get_source_db(alias) if not source_db: raise Exception(f"Can't find alias {alias} in config.") db_type = source_db.get("db_type") if db_type == SourceDatabase.mysql.value: from synch.reader.mysql import Mysql r = Mysql(alias) elif db_type == SourceDatabase.postgres.value: from synch.reader.postgres import Postgres r = Postgres(alias) else: raise NotImplementedError(f"Unsupported db_type {db_type}") _readers[alias] = r return r
def continuous_etl( alias: str, schema: str, tables_pk: Dict, tables_dict: Dict, last_msg_id, skip_error: bool, ): """ continuous etl from broker and insert into clickhouse """ global len_event global event_list global is_insert global last_insert_time broker = get_broker(alias) insert_interval = Settings.insert_interval() insert_num = Settings.insert_num() logger.info(f"start consumer for {alias}.{schema} success") signal.signal(signal.SIGINT, signal_handler) signal.signal(signal.SIGTERM, signal_handler) for msg_id, msg in broker.msgs(schema, last_msg_id=last_msg_id, count=Settings.insert_num(), block=insert_interval * 1000): if not msg_id and not msg: logger.info( f"Block {insert_interval} seconds timeout, insert current {len_event} events" ) if len_event > 0: is_insert = True alter_table = False query = None else: if is_stop: finish_continuous_etl(broker) continue else: alter_table = False query = None logger.debug(f"msg_id:{msg_id}, msg:{msg}") len_event += 1 event = msg table = event["table"] schema = event["schema"] action = event["action"] values = event["values"] if action == "query": alter_table = True query = values["query"] else: engine = tables_dict.get(table).get("clickhouse_engine") writer = get_writer(engine) event_list = writer.handle_event( tables_dict, tables_pk.get(table), schema, table, action, event_list, event, ) if (len_event == insert_num or time.time() - last_insert_time >= Settings.insert_interval()): is_insert = True if is_insert or alter_table: for table, v in event_list.items(): table_event_num = 0 pk = tables_pk.get(table) if isinstance(v, dict): writer = get_writer(ClickHouseEngine.merge_tree) insert = v.get("insert") delete = v.get("delete") if delete: delete_pks = list(delete.keys()) else: delete_pks = [] if insert: insert_events = list( sorted(insert.values(), key=lambda x: x.get("event_unixtime"))) else: insert_events = [] if skip_error: try: if delete_pks: writer.delete_events(schema, table, pk, delete_pks) if insert_events: writer.insert_events(schema, table, insert_events) except Exception as e: logger.error(f"insert event error,error: {e}", exc_info=True, stack_info=True) else: if delete_pks: writer.delete_events(schema, table, pk, delete_pks) if insert_events: writer.insert_events(schema, table, insert_events) table_event_num += len(delete_pks) table_event_num += len(insert_events) elif isinstance(v, list): table_event_num += len(v) writer = get_writer(ClickHouseEngine.collapsing_merge_tree) if v: if skip_error: try: writer.insert_events(schema, table, v) except Exception as e: logger.error(f"insert event error,error: {e}", exc_info=True, stack_info=True) else: writer.insert_events(schema, table, v) insert_log(alias, schema, table, table_event_num, 2) if alter_table: try: get_writer().execute(query) except Exception as e: logger.error(f"alter table error: {e}", exc_info=True, stack_info=True) if not skip_error: exit(-1) broker.commit(schema) logger.info(f"success commit {len_event} events") event_list = {} is_insert = False len_event = 0 last_insert_time = time.time() if is_stop: finish_continuous_etl(broker)
def test_parse_config_file(): Settings.init("synch.yaml")
def test_settings(): assert isinstance(Settings.debug(), bool) assert isinstance(Settings.insert_num(), int) assert isinstance(Settings.insert_interval(), int)
def get_mysql_database(): return Settings.get_source_db(alias_mysql).get("databases")[0].get( "database")
def get_postgres_database(): return Settings.get_source_db(alias_postgres).get("databases")[0].get( "database")
def _binlog_reading( self, only_tables, only_schemas, log_file, log_pos, server_id, skip_dmls, skip_delete_tables, skip_update_tables, ) -> Generator: stream = BinLogStreamReader( connection_settings=dict( host=self.host, port=self.port, user=self.user, passwd=self.password, ), resume_stream=True, blocking=True, server_id=server_id, only_tables=only_tables, only_schemas=only_schemas, only_events=self.only_events, log_file=log_file, log_pos=log_pos, fail_on_table_metadata_unavailable=True, slave_heartbeat=10, ) for binlog_event in stream: if isinstance(binlog_event, QueryEvent): schema = binlog_event.schema.decode() query = binlog_event.query.lower() if "alter" not in query: continue table, convent_sql = SqlConvert.to_clickhouse( schema, query, Settings.cluster_name()) if not convent_sql: continue event = { "table": table, "schema": schema, "action": "query", "values": { "query": convent_sql }, "event_unixtime": int(time.time() * 10**6), "action_seq": 0, } yield schema, None, event, stream.log_file, stream.log_pos else: schema = binlog_event.schema table = binlog_event.table skip_dml_table_name = f"{schema}.{table}" for row in binlog_event.rows: if isinstance(binlog_event, WriteRowsEvent): event = { "table": table, "schema": schema, "action": "insert", "values": row["values"], "event_unixtime": int(time.time() * 10**6), "action_seq": 2, } elif isinstance(binlog_event, UpdateRowsEvent): if "update" in skip_dmls or skip_dml_table_name in skip_update_tables: continue delete_event = { "table": table, "schema": schema, "action": "delete", "values": row["before_values"], "event_unixtime": int(time.time() * 10**6), "action_seq": 1, } yield binlog_event.schema, binlog_event.table, delete_event, stream.log_file, stream.log_pos event = { "table": table, "schema": schema, "action": "insert", "values": row["after_values"], "event_unixtime": int(time.time() * 10**6), "action_seq": 2, } elif isinstance(binlog_event, DeleteRowsEvent): if "delete" in skip_dmls or skip_dml_table_name in skip_delete_tables: continue event = { "table": table, "schema": schema, "action": "delete", "values": row["values"], "event_unixtime": int(time.time() * 10**6), "action_seq": 1, } else: return yield binlog_event.schema, binlog_event.table, event, stream.log_file, stream.log_pos