Uploaded image for project: 'MariaDB Server'
  1. MariaDB Server
  2. MDEV-8302

Duplicate key with parallel replication

Details

    Description

      Hello,

      we are having a random "duplicate key" problem with parallel replication. We were trying hard to reproduce it on our test servers as well, but currently it only happens on our live servers.

      The error we get is the following:

      150604  2:56:41 [ERROR] Slave SQL: Error 'Duplicate entry 'VerifyOrder_0-249' for key 'identry_idqueue'' on query. Default database: 'optovision'. Query: 'insert into `queueentry`(`identry`,`idqueue`,`entrydate`,`removedate`,`info`,`prio`,`reserved`,`ObjectContext`,`OriginHost`,`ReservingHost`,`UnreserveDate`,`ReservationCounter`,`OptimisticLockField`)values('VerifyOrder_0',249,'2015-06-04 02:56:40.687','2015-06-04 03:11:40.671','',0,null,'semaphore','172.16.185.22',null,null,0,0)', Gtid 0-3-454870355, Internal MariaDB error code: 1062
      150604  2:56:41 [Warning] Slave: Duplicate entry 'VerifyOrder_0-249' for key 'identry_idqueue' Error_code: 1062
      150604  2:56:41 [ERROR] Error running query, slave SQL thread aborted. Fix the problem, and restart the slave SQL thread with "SLAVE START". We stopped at log 'mysql-bin.007762' position 645226271; GTID position '77-3-552,0-3-454870353,3-3-2,2-3-469046,17-3-180,1-3-7'
      150604  2:56:41 [Note] Error reading relay log event: slave SQL thread was killed

      It is always the same table that is involved in this issue, and in all cases we analyzed in deep, it was the same value for the unique key to be written.

      The replication is based on one master and one slave, no restrictions on the data to be transferred, and no direct write requests to the slave. Once a day the slave is stopped for a short moment to synchronize the file system and take a backup on LVM level. At the moment we see no connection between this backup and the error (the point of time it happens seems to be independent)

      The table that is involved is defined as follows:

      CREATE TABLE `queueentry` (
      	`OID` INT(11) NOT NULL AUTO_INCREMENT,
      	`identry` VARCHAR(100) NOT NULL,
      	`entrydate` DATETIME NULL DEFAULT NULL,
      	`removedate` DATETIME NULL DEFAULT NULL,
      	`info` VARCHAR(8096) NULL DEFAULT NULL,
      	`prio` INT(11) NULL DEFAULT NULL,
      	`reserved` DATETIME NULL DEFAULT NULL,
      	`idqueue` INT(11) NOT NULL,
      	`ObjectContext` VARCHAR(200) NULL DEFAULT NULL,
      	`OriginHost` VARCHAR(200) NULL DEFAULT NULL,
      	`ReservingHost` VARCHAR(200) NULL DEFAULT NULL,
      	`UnreserveDate` DATETIME NULL DEFAULT NULL,
      	`ReservationCounter` INT(10) UNSIGNED NOT NULL DEFAULT '0',
      	`OptimisticLockField` INT(11) NULL DEFAULT NULL,
      	PRIMARY KEY (`OID`),
      	UNIQUE INDEX `identry_idqueue` (`identry`, `idqueue`),
      	INDEX `idqueue` (`idqueue`),
      	INDEX `entrydate_prio_reserved_idqueue` (`idqueue`, `reserved`, `prio`, `entrydate`),
      	INDEX `removedate` (`removedate`),
      	INDEX `entrydate` (`entrydate`),
      	INDEX `UnreserveDate_ReservationCounter` (`ReservationCounter`, `UnreserveDate`)
      )
      COLLATE='utf8_general_ci'
      ENGINE=InnoDB
      AUTO_INCREMENT=8164332;

      We analyzed the binlog of the master in those cases and found the following:

      • Just before this insert statement there are two transactions bound together for parallel replication.
      • Both of these transactions work on the same table as the insert does.
      • The second of these transactions deletes the entry with the unique key, that will be written by the statement showing the duplicate key error.

      That is the following two statements are bound together:

      insert into `queueentry`(`identry`,`idqueue`,`entrydate`,`removedate`,`info`,`prio`,`reserved`,`ObjectContext`,`OriginHost`,`ReservingHost`,`UnreserveDate`,`ReservationCounter`,`OptimisticLockField`)values('SME_PrintLabel_0',249,'2015-06-04 02:56:40.671','2015-06-04 03:11:40.656','',0,null,'semaphore','172.16.185.21',null,null,0,0)
      delete from `queueentry` where ((`OID` = 6363790) and (`OptimisticLockField` = 0))

      And then this next transaction crashes:

      insert into `queueentry`(`identry`,`idqueue`,`entrydate`,`removedate`,`info`,`prio`,`reserved`,`ObjectContext`,`OriginHost`,`ReservingHost`,`UnreserveDate`,`ReservationCounter`,`OptimisticLockField`)values('VerifyOrder_0',249,'2015-06-04 02:56:40.687','2015-06-04 03:11:40.671','',0,null,'semaphore','172.16.185.22',null,null,0,0)

      Here is the (I hope) most important part of the binlog of the master:

      #150604  2:56:40 server id 3  end_log_pos 645225814     GTID 0-3-454870353 cid=5620557227
      /*!100001 SET @@session.gtid_seq_no=454870353*//*!*/;
      BEGIN
      /*!*/;
      # at 645225814
      #150604  2:56:40 server id 3  end_log_pos 645225842     Intvar
      SET INSERT_ID=6363805/*!*/;
      # at 645225842
      #150604  2:56:40 server id 3  end_log_pos 645226244     Query   thread_id=19732488      exec_time=0     error_code=0
      SET TIMESTAMP=1433379400/*!*/;
      insert into `queueentry`(`identry`,`idqueue`,`entrydate`,`removedate`,`info`,`prio`,`reserved`,`ObjectContext`,`OriginHost`,`ReservingHost`,`UnreserveDate`,`ReservationCounter`,`OptimisticLockField`)values('SME_PrintLabel_0',249,'2015-06-04 02:56:40.671','2015-06-04 03:11:40.656','',0,null,'semaphore','172.16.185.21',null,null,0,0)
      /*!*/;
      # at 645226244
      #150604  2:56:40 server id 3  end_log_pos 645226271     Xid = 5620557224
      COMMIT/*!*/;
      # at 645226271
      #150604  2:56:40 server id 3  end_log_pos 645226311     GTID 0-3-454870354 cid=5620557227
      /*!100001 SET @@session.gtid_seq_no=454870354*//*!*/;
      BEGIN
      /*!*/;
      # at 645226311
      #150604  2:56:40 server id 3  end_log_pos 645226462     Query   thread_id=19732489      exec_time=0     error_code=0
      SET TIMESTAMP=1433379400/*!*/;
      delete from `queueentry` where ((`OID` = 6363790) and (`OptimisticLockField` = 0))
      /*!*/;
      # at 645226462
      #150604  2:56:40 server id 3  end_log_pos 645226489     Xid = 5620557228
      COMMIT/*!*/;
      # at 645226489
      #150604  2:56:40 server id 3  end_log_pos 645226527     GTID 0-3-454870355
      /*!100001 SET @@session.gtid_seq_no=454870355*//*!*/;
      BEGIN
      /*!*/;
      # at 645226527
      #150604  2:56:40 server id 3  end_log_pos 645226555     Intvar
      SET INSERT_ID=6363806/*!*/;
      # at 645226555
      #150604  2:56:40 server id 3  end_log_pos 645226954     Query   thread_id=19732488      exec_time=0     error_code=0
      SET TIMESTAMP=1433379400/*!*/;
      insert into `queueentry`(`identry`,`idqueue`,`entrydate`,`removedate`,`info`,`prio`,`reserved`,`ObjectContext`,`OriginHost`,`ReservingHost`,`UnreserveDate`,`ReservationCounter`,`OptimisticLockField`)values('VerifyOrder_0',249,'2015-06-04 02:56:40.687','2015-06-04 03:11:40.671','',0,null,'semaphore','172.16.185.22',null,null,0,0)
      /*!*/;
      # at 645226954
      #150604  2:56:40 server id 3  end_log_pos 645226981     Xid = 5620557238
      COMMIT/*!*/;
      # at 645226981

      The following additional observations we made:

      • The replication error only happens when the slave is at least one second behind the master. So it seems to us that there is somewhere a race condition at work which only happens when the slave is working very fast on the transactions from the master.
      • When this error happens, the command "stop slave" does not work any longer. We have to kill the database process, since some slave transactions are still waiting on each other.
      • After a restart of the database server we examine the table, and we cannot see the malicious key in the table any longer (even before the restart it is not there). We can restart the replication without further actions.
      • When we restart the replication, the slave is normally a few minutes behind the master. While catching up, we normally get the same error again.
      • As far as we can see, it is always the value 'VerifyOrder_0' that is causing the error, even though other values are written to that table as often as that value. We find this interesting, since it is - in alphabetical order - the largest value that is being stored in that table at the moment. I have only a rough understanding about indexes in databases, but I wonder if this could be related to an index reorganization due to the previous insert and delete statements.
      • The error vanishes if we set binlog_commit_wait_count=0 on the master (which is our current workaround).

      The variables on our slave are set as follows:

      'Variable_name','Value'
      'aria_block_size','8192'
      'aria_checkpoint_interval','30'
      'aria_checkpoint_log_activity','1048576'
      'aria_force_start_after_recovery_failures','0'
      'aria_group_commit','none'
      'aria_group_commit_interval','0'
      'aria_log_file_size','1073741824'
      'aria_log_purge_type','immediate'
      'aria_max_sort_file_size','9223372036853727232'
      'aria_page_checksum','ON'
      'aria_pagecache_age_threshold','300'
      'aria_pagecache_buffer_size','134217728'
      'aria_pagecache_division_limit','100'
      'aria_pagecache_file_hash_size','512'
      'aria_recover','NORMAL'
      'aria_repair_threads','1'
      'aria_sort_buffer_size','268434432'
      'aria_stats_method','nulls_unequal'
      'aria_sync_log_dir','NEWFILE'
      'aria_used_for_temp_tables','ON'
      'auto_increment_increment','1'
      'auto_increment_offset','1'
      'autocommit','ON'
      'automatic_sp_privileges','ON'
      'back_log','150'
      'basedir','/usr'
      'big_tables','OFF'
      'binlog_annotate_row_events','OFF'
      'binlog_cache_size','32768'
      'binlog_checksum','NONE'
      'binlog_commit_wait_count','0'
      'binlog_commit_wait_usec','100000'
      'binlog_direct_non_transactional_updates','OFF'
      'binlog_format','MIXED'
      'binlog_optimize_thread_scheduling','ON'
      'binlog_stmt_cache_size','32768'
      'bulk_insert_buffer_size','8388608'
      'character_set_client','utf8'
      'character_set_connection','utf8'
      'character_set_database','utf8'
      'character_set_filesystem','binary'
      'character_set_results','utf8'
      'character_set_server','utf8'
      'character_set_system','utf8'
      'character_sets_dir','/usr/share/mysql/charsets/'
      'collation_connection','utf8_general_ci'
      'collation_database','utf8_general_ci'
      'collation_server','utf8_unicode_ci'
      'completion_type','NO_CHAIN'
      'concurrent_insert','AUTO'
      'connect_timeout','10'
      'datadir','/var/lib/mysql/'
      'date_format','%Y-%m-%d'
      'datetime_format','%Y-%m-%d %H:%i:%s'
      'deadlock_search_depth_long','15'
      'deadlock_search_depth_short','4'
      'deadlock_timeout_long','50000000'
      'deadlock_timeout_short','10000'
      'debug_no_thread_alarm','OFF'
      'default_master_connection',''
      'default_regex_flags',''
      'default_storage_engine','InnoDB'
      'default_week_format','0'
      'delay_key_write','ON'
      'delayed_insert_limit','100'
      'delayed_insert_timeout','300'
      'delayed_queue_size','1000'
      'div_precision_increment','4'
      'error_count','0'
      'event_scheduler','OFF'
      'expensive_subquery_limit','100'
      'expire_logs_days','5'
      'external_user',''
      'extra_max_connections','1'
      'extra_port','0'
      'flush','OFF'
      'flush_time','0'
      'foreign_key_checks','ON'
      'ft_boolean_syntax','+ -><()~*:""&|'
      'ft_max_word_len','84'
      'ft_min_word_len','4'
      'ft_query_expansion_limit','20'
      'ft_stopword_file','(built-in)'
      'general_log','OFF'
      'general_log_file','optosql5.log'
      'group_concat_max_len','1024'
      'gtid_binlog_pos','0-3-481964408,3-3-2,2-3-504931'
      'gtid_binlog_state','0-5-479038918,0-3-481964408,3-3-2,2-3-504931'
      'gtid_current_pos','77-3-552,0-3-481964408,3-3-2,2-3-504931,17-3-180,1-3-7'
      'gtid_domain_id','0'
      'gtid_ignore_duplicates','OFF'
      'gtid_seq_no','0'
      'gtid_slave_pos','77-3-552,0-3-481964408,3-3-2,2-3-504931,17-3-180,1-3-7'
      'gtid_strict_mode','OFF'
      'have_compress','YES'
      'have_crypt','YES'
      'have_dynamic_loading','YES'
      'have_geometry','YES'
      'have_openssl','YES'
      'have_profiling','YES'
      'have_query_cache','YES'
      'have_rtree_keys','YES'
      'have_ssl','DISABLED'
      'have_symlink','DISABLED'
      'histogram_size','0'
      'histogram_type','SINGLE_PREC_HB'
      'host_cache_size','128'
      'hostname','optosql5'
      'identity','0'
      'ignore_builtin_innodb','OFF'
      'ignore_db_dirs',''
      'in_transaction','0'
      'init_connect',''
      'init_file',''
      'init_slave',''
      'innodb_adaptive_flushing','ON'
      'innodb_adaptive_flushing_lwm','10.000000'
      'innodb_adaptive_hash_index','ON'
      'innodb_adaptive_hash_index_partitions','1'
      'innodb_adaptive_max_sleep_delay','150000'
      'innodb_additional_mem_pool_size','8388608'
      'innodb_api_bk_commit_interval','5'
      'innodb_api_disable_rowlock','OFF'
      'innodb_api_enable_binlog','OFF'
      'innodb_api_enable_mdl','OFF'
      'innodb_api_trx_level','0'
      'innodb_autoextend_increment','64'
      'innodb_autoinc_lock_mode','1'
      'innodb_buffer_pool_dump_at_shutdown','OFF'
      'innodb_buffer_pool_dump_now','OFF'
      'innodb_buffer_pool_filename','ib_buffer_pool'
      'innodb_buffer_pool_instances','8'
      'innodb_buffer_pool_load_abort','OFF'
      'innodb_buffer_pool_load_at_startup','OFF'
      'innodb_buffer_pool_load_now','OFF'
      'innodb_buffer_pool_populate','OFF'
      'innodb_buffer_pool_size','34359738368'
      'innodb_change_buffer_max_size','25'
      'innodb_change_buffering','all'
      'innodb_checksum_algorithm','innodb'
      'innodb_checksums','ON'
      'innodb_cleaner_lsn_age_factor','high_checkpoint'
      'innodb_cmp_per_index_enabled','OFF'
      'innodb_commit_concurrency','0'
      'innodb_compression_failure_threshold_pct','5'
      'innodb_compression_level','6'
      'innodb_compression_pad_pct_max','50'
      'innodb_concurrency_tickets','5000'
      'innodb_corrupt_table_action','assert'
      'innodb_data_file_path','ibdata1:12M:autoextend'
      'innodb_data_home_dir',''
      'innodb_disable_sort_file_cache','OFF'
      'innodb_doublewrite','ON'
      'innodb_empty_free_list_algorithm','backoff'
      'innodb_fake_changes','OFF'
      'innodb_fast_shutdown','1'
      'innodb_file_format','Antelope'
      'innodb_file_format_check','ON'
      'innodb_file_format_max','Antelope'
      'innodb_file_per_table','ON'
      'innodb_flush_log_at_timeout','1'
      'innodb_flush_log_at_trx_commit','1'
      'innodb_flush_method',''
      'innodb_flush_neighbors','1'
      'innodb_flushing_avg_loops','30'
      'innodb_force_load_corrupted','OFF'
      'innodb_force_recovery','0'
      'innodb_foreground_preflush','exponential_backoff'
      'innodb_ft_aux_table',''
      'innodb_ft_cache_size','8000000'
      'innodb_ft_enable_diag_print','OFF'
      'innodb_ft_enable_stopword','ON'
      'innodb_ft_max_token_size','84'
      'innodb_ft_min_token_size','3'
      'innodb_ft_num_word_optimize','2000'
      'innodb_ft_result_cache_limit','2000000000'
      'innodb_ft_server_stopword_table',''
      'innodb_ft_sort_pll_degree','2'
      'innodb_ft_total_cache_size','640000000'
      'innodb_ft_user_stopword_table',''
      'innodb_io_capacity','200'
      'innodb_io_capacity_max','2000'
      'innodb_kill_idle_transaction','0'
      'innodb_large_prefix','OFF'
      'innodb_lock_wait_timeout','50'
      'innodb_locking_fake_changes','ON'
      'innodb_locks_unsafe_for_binlog','OFF'
      'innodb_log_arch_dir','./'
      'innodb_log_arch_expire_sec','0'
      'innodb_log_archive','OFF'
      'innodb_log_block_size','512'
      'innodb_log_buffer_size','8388608'
      'innodb_log_checksum_algorithm','innodb'
      'innodb_log_compressed_pages','ON'
      'innodb_log_file_size','104857600'
      'innodb_log_files_in_group','2'
      'innodb_log_group_home_dir','./'
      'innodb_lru_scan_depth','1024'
      'innodb_max_bitmap_file_size','104857600'
      'innodb_max_changed_pages','1000000'
      'innodb_max_dirty_pages_pct','75.000000'
      'innodb_max_dirty_pages_pct_lwm','0.001000'
      'innodb_max_purge_lag','0'
      'innodb_max_purge_lag_delay','0'
      'innodb_mirrored_log_groups','1'
      'innodb_monitor_disable',''
      'innodb_monitor_enable',''
      'innodb_monitor_reset',''
      'innodb_monitor_reset_all',''
      'innodb_old_blocks_pct','37'
      'innodb_old_blocks_time','1000'
      'innodb_online_alter_log_max_size','134217728'
      'innodb_open_files','512'
      'innodb_optimize_fulltext_only','OFF'
      'innodb_page_size','16384'
      'innodb_print_all_deadlocks','OFF'
      'innodb_purge_batch_size','300'
      'innodb_purge_threads','1'
      'innodb_random_read_ahead','OFF'
      'innodb_read_ahead_threshold','56'
      'innodb_read_io_threads','64'
      'innodb_read_only','OFF'
      'innodb_replication_delay','0'
      'innodb_rollback_on_timeout','OFF'
      'innodb_rollback_segments','128'
      'innodb_sched_priority_cleaner','19'
      'innodb_show_locks_held','10'
      'innodb_show_verbose_locks','0'
      'innodb_simulate_comp_failures','0'
      'innodb_sort_buffer_size','1048576'
      'innodb_spin_wait_delay','6'
      'innodb_stats_auto_recalc','ON'
      'innodb_stats_method','nulls_equal'
      'innodb_stats_modified_counter','0'
      'innodb_stats_on_metadata','OFF'
      'innodb_stats_persistent','ON'
      'innodb_stats_persistent_sample_pages','20'
      'innodb_stats_sample_pages','8'
      'innodb_stats_traditional','ON'
      'innodb_stats_transient_sample_pages','8'
      'innodb_status_output','OFF'
      'innodb_status_output_locks','OFF'
      'innodb_strict_mode','OFF'
      'innodb_support_xa','ON'
      'innodb_sync_array_size','1'
      'innodb_sync_spin_loops','30'
      'innodb_table_locks','ON'
      'innodb_thread_concurrency','0'
      'innodb_thread_sleep_delay','10000'
      'innodb_track_changed_pages','OFF'
      'innodb_undo_directory','.'
      'innodb_undo_logs','128'
      'innodb_undo_tablespaces','0'
      'innodb_use_atomic_writes','OFF'
      'innodb_use_fallocate','OFF'
      'innodb_use_global_flush_log_at_trx_commit','ON'
      'innodb_use_native_aio','ON'
      'innodb_use_stacktrace','OFF'
      'innodb_use_sys_malloc','ON'
      'innodb_version','5.6.22-71.0'
      'innodb_write_io_threads','64'
      'insert_id','0'
      'interactive_timeout','28800'
      'join_buffer_size','131072'
      'join_buffer_space_limit','2097152'
      'join_cache_level','2'
      'keep_files_on_create','OFF'
      'key_buffer_size','134217728'
      'key_cache_age_threshold','300'
      'key_cache_block_size','1024'
      'key_cache_division_limit','100'
      'key_cache_file_hash_size','512'
      'key_cache_segments','0'
      'large_files_support','ON'
      'large_page_size','0'
      'large_pages','OFF'
      'last_gtid',''
      'last_insert_id','0'
      'lc_messages','en_US'
      'lc_messages_dir',''
      'lc_time_names','en_US'
      'license','GPL'
      'local_infile','ON'
      'lock_wait_timeout','31536000'
      'locked_in_memory','OFF'
      'log_bin','ON'
      'log_bin_trust_function_creators','OFF'
      'log_error','/var/log/mysqld.log'
      'log_output','FILE'
      'log_queries_not_using_indexes','OFF'
      'log_slave_updates','ON'
      'log_slow_filter','admin,filesort,filesort_on_disk,full_join,full_scan,query_cache,query_cache_miss,tmp_table,tmp_table_on_disk'
      'log_slow_rate_limit','1'
      'log_slow_verbosity',''
      'log_warnings','1'
      'long_query_time','10.000000'
      'low_priority_updates','OFF'
      'lower_case_file_system','OFF'
      'lower_case_table_names','1'
      'master_verify_checksum','OFF'
      'max_allowed_packet','33554432'
      'max_binlog_cache_size','18446744073709547520'
      'max_binlog_size','1073741824'
      'max_binlog_stmt_cache_size','18446744073709547520'
      'max_connect_errors','100'
      'max_connections','500'
      'max_delayed_threads','20'
      'max_error_count','64'
      'max_heap_table_size','67108864'
      'max_insert_delayed_threads','20'
      'max_join_size','18446744073709551615'
      'max_length_for_sort_data','1024'
      'max_long_data_size','33554432'
      'max_prepared_stmt_count','16382'
      'max_relay_log_size','1073741824'
      'max_seeks_for_key','4294967295'
      'max_sort_length','1024'
      'max_sp_recursion_depth','0'
      'max_tmp_tables','32'
      'max_user_connections','0'
      'max_write_lock_count','4294967295'
      'metadata_locks_cache_size','1024'
      'metadata_locks_hash_instances','8'
      'min_examined_row_limit','0'
      'mrr_buffer_size','262144'
      'multi_range_count','256'
      'myisam_block_size','1024'
      'myisam_data_pointer_size','6'
      'myisam_max_sort_file_size','9223372036853727232'
      'myisam_mmap_size','18446744073709551615'
      'myisam_recover_options','DEFAULT'
      'myisam_repair_threads','1'
      'myisam_sort_buffer_size','134216704'
      'myisam_stats_method','nulls_unequal'
      'myisam_use_mmap','OFF'
      'net_buffer_length','16384'
      'net_read_timeout','30'
      'net_retry_count','10'
      'net_write_timeout','60'
      'old','OFF'
      'old_alter_table','OFF'
      'old_mode',''
      'old_passwords','OFF'
      'open_files_limit','500005'
      'optimizer_prune_level','1'
      'optimizer_search_depth','62'
      'optimizer_selectivity_sampling_limit','100'
      'optimizer_switch','index_merge=on,index_merge_union=on,index_merge_sort_union=on,index_merge_intersection=on,index_merge_sort_intersection=off,engine_condition_pushdown=off,index_condition_pushdown=on,derived_merge=on,derived_with_keys=on,firstmatch=on,loosescan=on,materialization=on,in_to_exists=on,semijoin=on,partial_match_rowid_merge=on,partial_match_table_scan=on,subquery_cache=on,mrr=off,mrr_cost_based=off,mrr_sort_keys=off,outer_join_with_cache=on,semijoin_with_cache=on,join_cache_incremental=on,join_cache_hashed=on,join_cache_bka=on,optimize_join_buffer_size=off,table_elimination=on,extended_keys=on,exists_to_in=on'
      'optimizer_use_condition_selectivity','1'
      'performance_schema','ON'
      'performance_schema_accounts_size','100'
      'performance_schema_digests_size','10000'
      'performance_schema_events_stages_history_long_size','10000'
      'performance_schema_events_stages_history_size','10'
      'performance_schema_events_statements_history_long_size','10000'
      'performance_schema_events_statements_history_size','10'
      'performance_schema_events_waits_history_long_size','10000'
      'performance_schema_events_waits_history_size','10'
      'performance_schema_hosts_size','100'
      'performance_schema_max_cond_classes','80'
      'performance_schema_max_cond_instances','400900'
      'performance_schema_max_file_classes','50'
      'performance_schema_max_file_handles','32768'
      'performance_schema_max_file_instances','2154'
      'performance_schema_max_mutex_classes','200'
      'performance_schema_max_mutex_instances','605000'
      'performance_schema_max_rwlock_classes','40'
      'performance_schema_max_rwlock_instances','202800'
      'performance_schema_max_socket_classes','10'
      'performance_schema_max_socket_instances','200020'
      'performance_schema_max_stage_classes','150'
      'performance_schema_max_statement_classes','180'
      'performance_schema_max_table_handles','800'
      'performance_schema_max_table_instances','12500'
      'performance_schema_max_thread_classes','50'
      'performance_schema_max_thread_instances','200100'
      'performance_schema_session_connect_attrs_size','512'
      'performance_schema_setup_actors_size','100'
      'performance_schema_setup_objects_size','100'
      'performance_schema_users_size','100'
      'pid_file','/var/lib/mysql/optosql5.pid'
      'plugin_dir','/usr/lib64/mysql/plugin/'
      'plugin_maturity','unknown'
      'port','3306'
      'preload_buffer_size','32768'
      'profiling','OFF'
      'profiling_history_size','15'
      'progress_report_time','5'
      'protocol_version','10'
      'proxy_user',''
      'pseudo_slave_mode','OFF'
      'pseudo_thread_id','189143'
      'query_alloc_block_size','8192'
      'query_cache_limit','1048576'
      'query_cache_min_res_unit','4096'
      'query_cache_size','67108864'
      'query_cache_strip_comments','OFF'
      'query_cache_type','ON'
      'query_cache_wlock_invalidate','OFF'
      'query_prealloc_size','8192'
      'rand_seed1','0'
      'rand_seed2','0'
      'range_alloc_block_size','4096'
      'read_buffer_size','131072'
      'read_only','ON'
      'read_rnd_buffer_size','262144'
      'relay_log',''
      'relay_log_index',''
      'relay_log_info_file','relay-log.info'
      'relay_log_purge','ON'
      'relay_log_recovery','OFF'
      'relay_log_space_limit','0'
      'replicate_annotate_row_events','OFF'
      'replicate_do_db',''
      'replicate_do_table',''
      'replicate_events_marked_for_skip','replicate'
      'replicate_ignore_db',''
      'replicate_ignore_table',''
      'replicate_wild_do_table',''
      'replicate_wild_ignore_table',''
      'report_host',''
      'report_password',''
      'report_port','3306'
      'report_user',''
      'rowid_merge_buff_size','8388608'
      'rpl_recovery_rank','0'
      'secure_auth','OFF'
      'secure_file_priv',''
      'server_id','5'
      'skip_external_locking','ON'
      'skip_name_resolve','ON'
      'skip_networking','OFF'
      'skip_replication','OFF'
      'skip_show_database','OFF'
      'slave_compressed_protocol','OFF'
      'slave_ddl_exec_mode','IDEMPOTENT'
      'slave_domain_parallel_threads','4'
      'slave_exec_mode','STRICT'
      'slave_load_tmpdir','/tmp'
      'slave_max_allowed_packet','1073741824'
      'slave_net_timeout','3600'
      'slave_parallel_max_queued','131072'
      'slave_parallel_threads','12'
      'slave_skip_errors','OFF'
      'slave_sql_verify_checksum','ON'
      'slave_transaction_retries','10'
      'slave_type_conversions',''
      'slow_launch_time','2'
      'slow_query_log','ON'
      'slow_query_log_file','optosql5-slow.log'
      'socket','/var/lib/mysql/mysql.sock'
      'sort_buffer_size','2097152'
      'sql_auto_is_null','OFF'
      'sql_big_selects','ON'
      'sql_buffer_result','OFF'
      'sql_log_bin','ON'
      'sql_log_off','OFF'
      'sql_mode',''
      'sql_notes','ON'
      'sql_quote_show_create','ON'
      'sql_safe_updates','OFF'
      'sql_select_limit','18446744073709551615'
      'sql_slave_skip_counter','0'
      'sql_warnings','OFF'
      'ssl_ca',''
      'ssl_capath',''
      'ssl_cert',''
      'ssl_cipher',''
      'ssl_crl',''
      'ssl_crlpath',''
      'ssl_key',''
      'storage_engine','InnoDB'
      'stored_program_cache','256'
      'sync_binlog','0'
      'sync_frm','ON'
      'sync_master_info','0'
      'sync_relay_log','0'
      'sync_relay_log_info','0'
      'system_time_zone','CEST'
      'table_definition_cache','400'
      'table_open_cache','512'
      'thread_cache_size','4'
      'thread_concurrency','10'
      'thread_handling','one-thread-per-connection'
      'thread_pool_idle_timeout','60'
      'thread_pool_max_threads','500'
      'thread_pool_oversubscribe','3'
      'thread_pool_size','16'
      'thread_pool_stall_limit','500'
      'thread_stack','294912'
      'time_format','%H:%i:%s'
      'time_zone','SYSTEM'
      'timed_mutexes','OFF'
      'timestamp','1434011008.405971'
      'tmp_table_size','134217728'
      'tmpdir','/tmp'
      'transaction_alloc_block_size','8192'
      'transaction_prealloc_size','4096'
      'tx_isolation','REPEATABLE-READ'
      'tx_read_only','OFF'
      'unique_checks','ON'
      'updatable_views_with_limit','YES'
      'use_stat_tables','NEVER'
      'userstat','OFF'
      'version','10.0.16-MariaDB-log'
      'version_comment','MariaDB Server'
      'version_compile_machine','x86_64'
      'version_compile_os','Linux'
      'version_malloc_library','bundled jemalloc'
      'wait_timeout','28800'
      'warning_count','0'

      The error occurs about once a week during normal operation.

      If you need further information, or if you want us to test something, please let me know.

      Best regards,

      Thomas Mischke

      Attachments

        Activity

          thomas.mischke Thomas Mischke created issue -
          thomas.mischke Thomas Mischke made changes -
          Field Original Value New Value
          Description Hello,

          we are having a random "duplicate key" problem with parallel replication. We were trying hard to reproduce it on our test servers as well, but currently it only happens on our live servers.

          The error we get is the following:

          {{
          150604 2:56:41 [ERROR] Slave SQL: Error 'Duplicate entry 'VerifyOrder_0-249' for key 'identry_idqueue'' on query. Default database: 'optovision'. Query: 'insert into `queueentry`(`identry`,`idqueue`,`entrydate`,`removedate`,`info`,`prio`,`reserved`,`ObjectContext`,`OriginHost`,`ReservingHost`,`UnreserveDate`,`ReservationCounter`,`OptimisticLockField`)values('VerifyOrder_0',249,'2015-06-04 02:56:40.687','2015-06-04 03:11:40.671','',0,null,'semaphore','172.16.185.22',null,null,0,0)', Gtid 0-3-454870355, Internal MariaDB error code: 1062
          150604 2:56:41 [Warning] Slave: Duplicate entry 'VerifyOrder_0-249' for key 'identry_idqueue' Error_code: 1062
          150604 2:56:41 [ERROR] Error running query, slave SQL thread aborted. Fix the problem, and restart the slave SQL thread with "SLAVE START". We stopped at log 'mysql-bin.007762' position 645226271; GTID position '77-3-552,0-3-454870353,3-3-2,2-3-469046,17-3-180,1-3-7'
          150604 2:56:41 [Note] Error reading relay log event: slave SQL thread was killed
          }}

          It is always the same table that is involved in this issue, and in all cases we analyzed in deep, it was the same value for the unique key to be written.

          The replication is based on one master and one slave, no restrictions on the data to be transferred, and no direct write requests to the slave. Once a day the slave is stopped for a short moment to synchronize the file system and take a backup on LVM level. At the moment we see no connection between this backup and the error (the point of time it happens seems to be independent)

          The table that is involved is defined as follows:

          {{
          CREATE TABLE `queueentry` (
          `OID` INT(11) NOT NULL AUTO_INCREMENT,
          `identry` VARCHAR(100) NOT NULL,
          `entrydate` DATETIME NULL DEFAULT NULL,
          `removedate` DATETIME NULL DEFAULT NULL,
          `info` VARCHAR(8096) NULL DEFAULT NULL,
          `prio` INT(11) NULL DEFAULT NULL,
          `reserved` DATETIME NULL DEFAULT NULL,
          `idqueue` INT(11) NOT NULL,
          `ObjectContext` VARCHAR(200) NULL DEFAULT NULL,
          `OriginHost` VARCHAR(200) NULL DEFAULT NULL,
          `ReservingHost` VARCHAR(200) NULL DEFAULT NULL,
          `UnreserveDate` DATETIME NULL DEFAULT NULL,
          `ReservationCounter` INT(10) UNSIGNED NOT NULL DEFAULT '0',
          `OptimisticLockField` INT(11) NULL DEFAULT NULL,
          PRIMARY KEY (`OID`),
          UNIQUE INDEX `identry_idqueue` (`identry`, `idqueue`),
          INDEX `idqueue` (`idqueue`),
          INDEX `entrydate_prio_reserved_idqueue` (`idqueue`, `reserved`, `prio`, `entrydate`),
          INDEX `removedate` (`removedate`),
          INDEX `entrydate` (`entrydate`),
          INDEX `UnreserveDate_ReservationCounter` (`ReservationCounter`, `UnreserveDate`)
          )
          COLLATE='utf8_general_ci'
          ENGINE=InnoDB
          AUTO_INCREMENT=8164332;
          }}

          We analyzed the binlog of the master in those cases and found the following:
          * Just before this insert statement there are two transactions bound together for parallel replication.
          * Both of these transactions work on the same table as the insert does.
          * The second of these transactions deletes the entry with the unique key, that will be written by the statement showing the duplicate key error.

          That is the following two statements are bound together:
          {{
          insert into `queueentry`(`identry`,`idqueue`,`entrydate`,`removedate`,`info`,`prio`,`reserved`,`ObjectContext`,`OriginHost`,`ReservingHost`,`UnreserveDate`,`ReservationCounter`,`OptimisticLockField`)values('SME_PrintLabel_0',249,'2015-06-04 02:56:40.671','2015-06-04 03:11:40.656','',0,null,'semaphore','172.16.185.21',null,null,0,0)
          delete from `queueentry` where ((`OID` = 6363790) and (`OptimisticLockField` = 0))
          }}

          And then this next transaction crashes:
          {{
          insert into `queueentry`(`identry`,`idqueue`,`entrydate`,`removedate`,`info`,`prio`,`reserved`,`ObjectContext`,`OriginHost`,`ReservingHost`,`UnreserveDate`,`ReservationCounter`,`OptimisticLockField`)values('VerifyOrder_0',249,'2015-06-04 02:56:40.687','2015-06-04 03:11:40.671','',0,null,'semaphore','172.16.185.22',null,null,0,0)
          }}

          Here is the (I hope) most important part of the binlog of the master:

          {{
          #150604 2:56:40 server id 3 end_log_pos 645225814 GTID 0-3-454870353 cid=5620557227
          /*!100001 SET @@session.gtid_seq_no=454870353*//*!*/;
          BEGIN
          /*!*/;
          # at 645225814
          #150604 2:56:40 server id 3 end_log_pos 645225842 Intvar
          SET INSERT_ID=6363805/*!*/;
          # at 645225842
          #150604 2:56:40 server id 3 end_log_pos 645226244 Query thread_id=19732488 exec_time=0 error_code=0
          SET TIMESTAMP=1433379400/*!*/;
          insert into `queueentry`(`identry`,`idqueue`,`entrydate`,`removedate`,`info`,`prio`,`reserved`,`ObjectContext`,`OriginHost`,`ReservingHost`,`UnreserveDate`,`ReservationCounter`,`OptimisticLockField`)values('SME_PrintLabel_0',249,'2015-06-04 02:56:40.671','2015-06-04 03:11:40.656','',0,null,'semaphore','172.16.185.21',null,null,0,0)
          /*!*/;
          # at 645226244
          #150604 2:56:40 server id 3 end_log_pos 645226271 Xid = 5620557224
          COMMIT/*!*/;
          # at 645226271
          #150604 2:56:40 server id 3 end_log_pos 645226311 GTID 0-3-454870354 cid=5620557227
          /*!100001 SET @@session.gtid_seq_no=454870354*//*!*/;
          BEGIN
          /*!*/;
          # at 645226311
          #150604 2:56:40 server id 3 end_log_pos 645226462 Query thread_id=19732489 exec_time=0 error_code=0
          SET TIMESTAMP=1433379400/*!*/;
          delete from `queueentry` where ((`OID` = 6363790) and (`OptimisticLockField` = 0))
          /*!*/;
          # at 645226462
          #150604 2:56:40 server id 3 end_log_pos 645226489 Xid = 5620557228
          COMMIT/*!*/;
          # at 645226489
          #150604 2:56:40 server id 3 end_log_pos 645226527 GTID 0-3-454870355
          /*!100001 SET @@session.gtid_seq_no=454870355*//*!*/;
          BEGIN
          /*!*/;
          # at 645226527
          #150604 2:56:40 server id 3 end_log_pos 645226555 Intvar
          SET INSERT_ID=6363806/*!*/;
          # at 645226555
          #150604 2:56:40 server id 3 end_log_pos 645226954 Query thread_id=19732488 exec_time=0 error_code=0
          SET TIMESTAMP=1433379400/*!*/;
          insert into `queueentry`(`identry`,`idqueue`,`entrydate`,`removedate`,`info`,`prio`,`reserved`,`ObjectContext`,`OriginHost`,`ReservingHost`,`UnreserveDate`,`ReservationCounter`,`OptimisticLockField`)values('VerifyOrder_0',249,'2015-06-04 02:56:40.687','2015-06-04 03:11:40.671','',0,null,'semaphore','172.16.185.22',null,null,0,0)
          /*!*/;
          # at 645226954
          #150604 2:56:40 server id 3 end_log_pos 645226981 Xid = 5620557238
          COMMIT/*!*/;
          # at 645226981
          }}

          The following additional observations we made:
          * The replication error only happens when the slave is at least one second behind the master. So it seems to us that there is somewhere a race condition at work which only happens when the slave is working very fast on the transactions from the master.
          * When this error happens, the command "stop slave" does not work any longer. We have to kill the database process, since some slave transactions are still waiting on each other.
          * After a restart of the database server we examine the table, and we cannot see the malicious key in the table any longer (even before the restart it is not there). We can restart the replication without further actions.
          * When we restart the replication, the slave is normally a few minutes behind the master. While catching up, we normally get the same error again.
          * As far as we can see, it is always the value 'VerifyOrder_0' that is causing the error, even though other values are written to that table as often as that value. We find this interesting, since it is - in alphabetical order - the largest value that is being stored in that table at the moment. I have only a rough understanding about indexes in databases, but I wonder if this could be related to an index reorganization due to the previous insert and delete statements.
          * The error vanishes if we set binlog_commit_wait_count=0 on the master (which is our current workaround).

          The variables on our slave are set as follows:

          {{
          'Variable_name','Value'
          'aria_block_size','8192'
          'aria_checkpoint_interval','30'
          'aria_checkpoint_log_activity','1048576'
          'aria_force_start_after_recovery_failures','0'
          'aria_group_commit','none'
          'aria_group_commit_interval','0'
          'aria_log_file_size','1073741824'
          'aria_log_purge_type','immediate'
          'aria_max_sort_file_size','9223372036853727232'
          'aria_page_checksum','ON'
          'aria_pagecache_age_threshold','300'
          'aria_pagecache_buffer_size','134217728'
          'aria_pagecache_division_limit','100'
          'aria_pagecache_file_hash_size','512'
          'aria_recover','NORMAL'
          'aria_repair_threads','1'
          'aria_sort_buffer_size','268434432'
          'aria_stats_method','nulls_unequal'
          'aria_sync_log_dir','NEWFILE'
          'aria_used_for_temp_tables','ON'
          'auto_increment_increment','1'
          'auto_increment_offset','1'
          'autocommit','ON'
          'automatic_sp_privileges','ON'
          'back_log','150'
          'basedir','/usr'
          'big_tables','OFF'
          'binlog_annotate_row_events','OFF'
          'binlog_cache_size','32768'
          'binlog_checksum','NONE'
          'binlog_commit_wait_count','0'
          'binlog_commit_wait_usec','100000'
          'binlog_direct_non_transactional_updates','OFF'
          'binlog_format','MIXED'
          'binlog_optimize_thread_scheduling','ON'
          'binlog_stmt_cache_size','32768'
          'bulk_insert_buffer_size','8388608'
          'character_set_client','utf8'
          'character_set_connection','utf8'
          'character_set_database','utf8'
          'character_set_filesystem','binary'
          'character_set_results','utf8'
          'character_set_server','utf8'
          'character_set_system','utf8'
          'character_sets_dir','/usr/share/mysql/charsets/'
          'collation_connection','utf8_general_ci'
          'collation_database','utf8_general_ci'
          'collation_server','utf8_unicode_ci'
          'completion_type','NO_CHAIN'
          'concurrent_insert','AUTO'
          'connect_timeout','10'
          'datadir','/var/lib/mysql/'
          'date_format','%Y-%m-%d'
          'datetime_format','%Y-%m-%d %H:%i:%s'
          'deadlock_search_depth_long','15'
          'deadlock_search_depth_short','4'
          'deadlock_timeout_long','50000000'
          'deadlock_timeout_short','10000'
          'debug_no_thread_alarm','OFF'
          'default_master_connection',''
          'default_regex_flags',''
          'default_storage_engine','InnoDB'
          'default_week_format','0'
          'delay_key_write','ON'
          'delayed_insert_limit','100'
          'delayed_insert_timeout','300'
          'delayed_queue_size','1000'
          'div_precision_increment','4'
          'error_count','0'
          'event_scheduler','OFF'
          'expensive_subquery_limit','100'
          'expire_logs_days','5'
          'external_user',''
          'extra_max_connections','1'
          'extra_port','0'
          'flush','OFF'
          'flush_time','0'
          'foreign_key_checks','ON'
          'ft_boolean_syntax','+ -><()~*:""&|'
          'ft_max_word_len','84'
          'ft_min_word_len','4'
          'ft_query_expansion_limit','20'
          'ft_stopword_file','(built-in)'
          'general_log','OFF'
          'general_log_file','optosql5.log'
          'group_concat_max_len','1024'
          'gtid_binlog_pos','0-3-481964408,3-3-2,2-3-504931'
          'gtid_binlog_state','0-5-479038918,0-3-481964408,3-3-2,2-3-504931'
          'gtid_current_pos','77-3-552,0-3-481964408,3-3-2,2-3-504931,17-3-180,1-3-7'
          'gtid_domain_id','0'
          'gtid_ignore_duplicates','OFF'
          'gtid_seq_no','0'
          'gtid_slave_pos','77-3-552,0-3-481964408,3-3-2,2-3-504931,17-3-180,1-3-7'
          'gtid_strict_mode','OFF'
          'have_compress','YES'
          'have_crypt','YES'
          'have_dynamic_loading','YES'
          'have_geometry','YES'
          'have_openssl','YES'
          'have_profiling','YES'
          'have_query_cache','YES'
          'have_rtree_keys','YES'
          'have_ssl','DISABLED'
          'have_symlink','DISABLED'
          'histogram_size','0'
          'histogram_type','SINGLE_PREC_HB'
          'host_cache_size','128'
          'hostname','optosql5'
          'identity','0'
          'ignore_builtin_innodb','OFF'
          'ignore_db_dirs',''
          'in_transaction','0'
          'init_connect',''
          'init_file',''
          'init_slave',''
          'innodb_adaptive_flushing','ON'
          'innodb_adaptive_flushing_lwm','10.000000'
          'innodb_adaptive_hash_index','ON'
          'innodb_adaptive_hash_index_partitions','1'
          'innodb_adaptive_max_sleep_delay','150000'
          'innodb_additional_mem_pool_size','8388608'
          'innodb_api_bk_commit_interval','5'
          'innodb_api_disable_rowlock','OFF'
          'innodb_api_enable_binlog','OFF'
          'innodb_api_enable_mdl','OFF'
          'innodb_api_trx_level','0'
          'innodb_autoextend_increment','64'
          'innodb_autoinc_lock_mode','1'
          'innodb_buffer_pool_dump_at_shutdown','OFF'
          'innodb_buffer_pool_dump_now','OFF'
          'innodb_buffer_pool_filename','ib_buffer_pool'
          'innodb_buffer_pool_instances','8'
          'innodb_buffer_pool_load_abort','OFF'
          'innodb_buffer_pool_load_at_startup','OFF'
          'innodb_buffer_pool_load_now','OFF'
          'innodb_buffer_pool_populate','OFF'
          'innodb_buffer_pool_size','34359738368'
          'innodb_change_buffer_max_size','25'
          'innodb_change_buffering','all'
          'innodb_checksum_algorithm','innodb'
          'innodb_checksums','ON'
          'innodb_cleaner_lsn_age_factor','high_checkpoint'
          'innodb_cmp_per_index_enabled','OFF'
          'innodb_commit_concurrency','0'
          'innodb_compression_failure_threshold_pct','5'
          'innodb_compression_level','6'
          'innodb_compression_pad_pct_max','50'
          'innodb_concurrency_tickets','5000'
          'innodb_corrupt_table_action','assert'
          'innodb_data_file_path','ibdata1:12M:autoextend'
          'innodb_data_home_dir',''
          'innodb_disable_sort_file_cache','OFF'
          'innodb_doublewrite','ON'
          'innodb_empty_free_list_algorithm','backoff'
          'innodb_fake_changes','OFF'
          'innodb_fast_shutdown','1'
          'innodb_file_format','Antelope'
          'innodb_file_format_check','ON'
          'innodb_file_format_max','Antelope'
          'innodb_file_per_table','ON'
          'innodb_flush_log_at_timeout','1'
          'innodb_flush_log_at_trx_commit','1'
          'innodb_flush_method',''
          'innodb_flush_neighbors','1'
          'innodb_flushing_avg_loops','30'
          'innodb_force_load_corrupted','OFF'
          'innodb_force_recovery','0'
          'innodb_foreground_preflush','exponential_backoff'
          'innodb_ft_aux_table',''
          'innodb_ft_cache_size','8000000'
          'innodb_ft_enable_diag_print','OFF'
          'innodb_ft_enable_stopword','ON'
          'innodb_ft_max_token_size','84'
          'innodb_ft_min_token_size','3'
          'innodb_ft_num_word_optimize','2000'
          'innodb_ft_result_cache_limit','2000000000'
          'innodb_ft_server_stopword_table',''
          'innodb_ft_sort_pll_degree','2'
          'innodb_ft_total_cache_size','640000000'
          'innodb_ft_user_stopword_table',''
          'innodb_io_capacity','200'
          'innodb_io_capacity_max','2000'
          'innodb_kill_idle_transaction','0'
          'innodb_large_prefix','OFF'
          'innodb_lock_wait_timeout','50'
          'innodb_locking_fake_changes','ON'
          'innodb_locks_unsafe_for_binlog','OFF'
          'innodb_log_arch_dir','./'
          'innodb_log_arch_expire_sec','0'
          'innodb_log_archive','OFF'
          'innodb_log_block_size','512'
          'innodb_log_buffer_size','8388608'
          'innodb_log_checksum_algorithm','innodb'
          'innodb_log_compressed_pages','ON'
          'innodb_log_file_size','104857600'
          'innodb_log_files_in_group','2'
          'innodb_log_group_home_dir','./'
          'innodb_lru_scan_depth','1024'
          'innodb_max_bitmap_file_size','104857600'
          'innodb_max_changed_pages','1000000'
          'innodb_max_dirty_pages_pct','75.000000'
          'innodb_max_dirty_pages_pct_lwm','0.001000'
          'innodb_max_purge_lag','0'
          'innodb_max_purge_lag_delay','0'
          'innodb_mirrored_log_groups','1'
          'innodb_monitor_disable',''
          'innodb_monitor_enable',''
          'innodb_monitor_reset',''
          'innodb_monitor_reset_all',''
          'innodb_old_blocks_pct','37'
          'innodb_old_blocks_time','1000'
          'innodb_online_alter_log_max_size','134217728'
          'innodb_open_files','512'
          'innodb_optimize_fulltext_only','OFF'
          'innodb_page_size','16384'
          'innodb_print_all_deadlocks','OFF'
          'innodb_purge_batch_size','300'
          'innodb_purge_threads','1'
          'innodb_random_read_ahead','OFF'
          'innodb_read_ahead_threshold','56'
          'innodb_read_io_threads','64'
          'innodb_read_only','OFF'
          'innodb_replication_delay','0'
          'innodb_rollback_on_timeout','OFF'
          'innodb_rollback_segments','128'
          'innodb_sched_priority_cleaner','19'
          'innodb_show_locks_held','10'
          'innodb_show_verbose_locks','0'
          'innodb_simulate_comp_failures','0'
          'innodb_sort_buffer_size','1048576'
          'innodb_spin_wait_delay','6'
          'innodb_stats_auto_recalc','ON'
          'innodb_stats_method','nulls_equal'
          'innodb_stats_modified_counter','0'
          'innodb_stats_on_metadata','OFF'
          'innodb_stats_persistent','ON'
          'innodb_stats_persistent_sample_pages','20'
          'innodb_stats_sample_pages','8'
          'innodb_stats_traditional','ON'
          'innodb_stats_transient_sample_pages','8'
          'innodb_status_output','OFF'
          'innodb_status_output_locks','OFF'
          'innodb_strict_mode','OFF'
          'innodb_support_xa','ON'
          'innodb_sync_array_size','1'
          'innodb_sync_spin_loops','30'
          'innodb_table_locks','ON'
          'innodb_thread_concurrency','0'
          'innodb_thread_sleep_delay','10000'
          'innodb_track_changed_pages','OFF'
          'innodb_undo_directory','.'
          'innodb_undo_logs','128'
          'innodb_undo_tablespaces','0'
          'innodb_use_atomic_writes','OFF'
          'innodb_use_fallocate','OFF'
          'innodb_use_global_flush_log_at_trx_commit','ON'
          'innodb_use_native_aio','ON'
          'innodb_use_stacktrace','OFF'
          'innodb_use_sys_malloc','ON'
          'innodb_version','5.6.22-71.0'
          'innodb_write_io_threads','64'
          'insert_id','0'
          'interactive_timeout','28800'
          'join_buffer_size','131072'
          'join_buffer_space_limit','2097152'
          'join_cache_level','2'
          'keep_files_on_create','OFF'
          'key_buffer_size','134217728'
          'key_cache_age_threshold','300'
          'key_cache_block_size','1024'
          'key_cache_division_limit','100'
          'key_cache_file_hash_size','512'
          'key_cache_segments','0'
          'large_files_support','ON'
          'large_page_size','0'
          'large_pages','OFF'
          'last_gtid',''
          'last_insert_id','0'
          'lc_messages','en_US'
          'lc_messages_dir',''
          'lc_time_names','en_US'
          'license','GPL'
          'local_infile','ON'
          'lock_wait_timeout','31536000'
          'locked_in_memory','OFF'
          'log_bin','ON'
          'log_bin_trust_function_creators','OFF'
          'log_error','/var/log/mysqld.log'
          'log_output','FILE'
          'log_queries_not_using_indexes','OFF'
          'log_slave_updates','ON'
          'log_slow_filter','admin,filesort,filesort_on_disk,full_join,full_scan,query_cache,query_cache_miss,tmp_table,tmp_table_on_disk'
          'log_slow_rate_limit','1'
          'log_slow_verbosity',''
          'log_warnings','1'
          'long_query_time','10.000000'
          'low_priority_updates','OFF'
          'lower_case_file_system','OFF'
          'lower_case_table_names','1'
          'master_verify_checksum','OFF'
          'max_allowed_packet','33554432'
          'max_binlog_cache_size','18446744073709547520'
          'max_binlog_size','1073741824'
          'max_binlog_stmt_cache_size','18446744073709547520'
          'max_connect_errors','100'
          'max_connections','500'
          'max_delayed_threads','20'
          'max_error_count','64'
          'max_heap_table_size','67108864'
          'max_insert_delayed_threads','20'
          'max_join_size','18446744073709551615'
          'max_length_for_sort_data','1024'
          'max_long_data_size','33554432'
          'max_prepared_stmt_count','16382'
          'max_relay_log_size','1073741824'
          'max_seeks_for_key','4294967295'
          'max_sort_length','1024'
          'max_sp_recursion_depth','0'
          'max_tmp_tables','32'
          'max_user_connections','0'
          'max_write_lock_count','4294967295'
          'metadata_locks_cache_size','1024'
          'metadata_locks_hash_instances','8'
          'min_examined_row_limit','0'
          'mrr_buffer_size','262144'
          'multi_range_count','256'
          'myisam_block_size','1024'
          'myisam_data_pointer_size','6'
          'myisam_max_sort_file_size','9223372036853727232'
          'myisam_mmap_size','18446744073709551615'
          'myisam_recover_options','DEFAULT'
          'myisam_repair_threads','1'
          'myisam_sort_buffer_size','134216704'
          'myisam_stats_method','nulls_unequal'
          'myisam_use_mmap','OFF'
          'net_buffer_length','16384'
          'net_read_timeout','30'
          'net_retry_count','10'
          'net_write_timeout','60'
          'old','OFF'
          'old_alter_table','OFF'
          'old_mode',''
          'old_passwords','OFF'
          'open_files_limit','500005'
          'optimizer_prune_level','1'
          'optimizer_search_depth','62'
          'optimizer_selectivity_sampling_limit','100'
          'optimizer_switch','index_merge=on,index_merge_union=on,index_merge_sort_union=on,index_merge_intersection=on,index_merge_sort_intersection=off,engine_condition_pushdown=off,index_condition_pushdown=on,derived_merge=on,derived_with_keys=on,firstmatch=on,loosescan=on,materialization=on,in_to_exists=on,semijoin=on,partial_match_rowid_merge=on,partial_match_table_scan=on,subquery_cache=on,mrr=off,mrr_cost_based=off,mrr_sort_keys=off,outer_join_with_cache=on,semijoin_with_cache=on,join_cache_incremental=on,join_cache_hashed=on,join_cache_bka=on,optimize_join_buffer_size=off,table_elimination=on,extended_keys=on,exists_to_in=on'
          'optimizer_use_condition_selectivity','1'
          'performance_schema','ON'
          'performance_schema_accounts_size','100'
          'performance_schema_digests_size','10000'
          'performance_schema_events_stages_history_long_size','10000'
          'performance_schema_events_stages_history_size','10'
          'performance_schema_events_statements_history_long_size','10000'
          'performance_schema_events_statements_history_size','10'
          'performance_schema_events_waits_history_long_size','10000'
          'performance_schema_events_waits_history_size','10'
          'performance_schema_hosts_size','100'
          'performance_schema_max_cond_classes','80'
          'performance_schema_max_cond_instances','400900'
          'performance_schema_max_file_classes','50'
          'performance_schema_max_file_handles','32768'
          'performance_schema_max_file_instances','2154'
          'performance_schema_max_mutex_classes','200'
          'performance_schema_max_mutex_instances','605000'
          'performance_schema_max_rwlock_classes','40'
          'performance_schema_max_rwlock_instances','202800'
          'performance_schema_max_socket_classes','10'
          'performance_schema_max_socket_instances','200020'
          'performance_schema_max_stage_classes','150'
          'performance_schema_max_statement_classes','180'
          'performance_schema_max_table_handles','800'
          'performance_schema_max_table_instances','12500'
          'performance_schema_max_thread_classes','50'
          'performance_schema_max_thread_instances','200100'
          'performance_schema_session_connect_attrs_size','512'
          'performance_schema_setup_actors_size','100'
          'performance_schema_setup_objects_size','100'
          'performance_schema_users_size','100'
          'pid_file','/var/lib/mysql/optosql5.pid'
          'plugin_dir','/usr/lib64/mysql/plugin/'
          'plugin_maturity','unknown'
          'port','3306'
          'preload_buffer_size','32768'
          'profiling','OFF'
          'profiling_history_size','15'
          'progress_report_time','5'
          'protocol_version','10'
          'proxy_user',''
          'pseudo_slave_mode','OFF'
          'pseudo_thread_id','189143'
          'query_alloc_block_size','8192'
          'query_cache_limit','1048576'
          'query_cache_min_res_unit','4096'
          'query_cache_size','67108864'
          'query_cache_strip_comments','OFF'
          'query_cache_type','ON'
          'query_cache_wlock_invalidate','OFF'
          'query_prealloc_size','8192'
          'rand_seed1','0'
          'rand_seed2','0'
          'range_alloc_block_size','4096'
          'read_buffer_size','131072'
          'read_only','ON'
          'read_rnd_buffer_size','262144'
          'relay_log',''
          'relay_log_index',''
          'relay_log_info_file','relay-log.info'
          'relay_log_purge','ON'
          'relay_log_recovery','OFF'
          'relay_log_space_limit','0'
          'replicate_annotate_row_events','OFF'
          'replicate_do_db',''
          'replicate_do_table',''
          'replicate_events_marked_for_skip','replicate'
          'replicate_ignore_db',''
          'replicate_ignore_table',''
          'replicate_wild_do_table',''
          'replicate_wild_ignore_table',''
          'report_host',''
          'report_password',''
          'report_port','3306'
          'report_user',''
          'rowid_merge_buff_size','8388608'
          'rpl_recovery_rank','0'
          'secure_auth','OFF'
          'secure_file_priv',''
          'server_id','5'
          'skip_external_locking','ON'
          'skip_name_resolve','ON'
          'skip_networking','OFF'
          'skip_replication','OFF'
          'skip_show_database','OFF'
          'slave_compressed_protocol','OFF'
          'slave_ddl_exec_mode','IDEMPOTENT'
          'slave_domain_parallel_threads','4'
          'slave_exec_mode','STRICT'
          'slave_load_tmpdir','/tmp'
          'slave_max_allowed_packet','1073741824'
          'slave_net_timeout','3600'
          'slave_parallel_max_queued','131072'
          'slave_parallel_threads','12'
          'slave_skip_errors','OFF'
          'slave_sql_verify_checksum','ON'
          'slave_transaction_retries','10'
          'slave_type_conversions',''
          'slow_launch_time','2'
          'slow_query_log','ON'
          'slow_query_log_file','optosql5-slow.log'
          'socket','/var/lib/mysql/mysql.sock'
          'sort_buffer_size','2097152'
          'sql_auto_is_null','OFF'
          'sql_big_selects','ON'
          'sql_buffer_result','OFF'
          'sql_log_bin','ON'
          'sql_log_off','OFF'
          'sql_mode',''
          'sql_notes','ON'
          'sql_quote_show_create','ON'
          'sql_safe_updates','OFF'
          'sql_select_limit','18446744073709551615'
          'sql_slave_skip_counter','0'
          'sql_warnings','OFF'
          'ssl_ca',''
          'ssl_capath',''
          'ssl_cert',''
          'ssl_cipher',''
          'ssl_crl',''
          'ssl_crlpath',''
          'ssl_key',''
          'storage_engine','InnoDB'
          'stored_program_cache','256'
          'sync_binlog','0'
          'sync_frm','ON'
          'sync_master_info','0'
          'sync_relay_log','0'
          'sync_relay_log_info','0'
          'system_time_zone','CEST'
          'table_definition_cache','400'
          'table_open_cache','512'
          'thread_cache_size','4'
          'thread_concurrency','10'
          'thread_handling','one-thread-per-connection'
          'thread_pool_idle_timeout','60'
          'thread_pool_max_threads','500'
          'thread_pool_oversubscribe','3'
          'thread_pool_size','16'
          'thread_pool_stall_limit','500'
          'thread_stack','294912'
          'time_format','%H:%i:%s'
          'time_zone','SYSTEM'
          'timed_mutexes','OFF'
          'timestamp','1434011008.405971'
          'tmp_table_size','134217728'
          'tmpdir','/tmp'
          'transaction_alloc_block_size','8192'
          'transaction_prealloc_size','4096'
          'tx_isolation','REPEATABLE-READ'
          'tx_read_only','OFF'
          'unique_checks','ON'
          'updatable_views_with_limit','YES'
          'use_stat_tables','NEVER'
          'userstat','OFF'
          'version','10.0.16-MariaDB-log'
          'version_comment','MariaDB Server'
          'version_compile_machine','x86_64'
          'version_compile_os','Linux'
          'version_malloc_library','bundled jemalloc'
          'wait_timeout','28800'
          'warning_count','0'
          }}

          The error occurs about once a week during normal operation.

          If you need further information, or if you want us to test something, please let me know.

          Best regards,

          Thomas Mischke
          Hello,

          we are having a random "duplicate key" problem with parallel replication. We were trying hard to reproduce it on our test servers as well, but currently it only happens on our live servers.

          The error we get is the following:

          {{150604 2:56:41 [ERROR] Slave SQL: Error 'Duplicate entry 'VerifyOrder_0-249' for key 'identry_idqueue'' on query. Default database: 'optovision'. Query: 'insert into `queueentry`(`identry`,`idqueue`,`entrydate`,`removedate`,`info`,`prio`,`reserved`,`ObjectContext`,`OriginHost`,`ReservingHost`,`UnreserveDate`,`ReservationCounter`,`OptimisticLockField`)values('VerifyOrder_0',249,'2015-06-04 02:56:40.687','2015-06-04 03:11:40.671','',0,null,'semaphore','172.16.185.22',null,null,0,0)', Gtid 0-3-454870355, Internal MariaDB error code: 1062
          150604 2:56:41 [Warning] Slave: Duplicate entry 'VerifyOrder_0-249' for key 'identry_idqueue' Error_code: 1062
          150604 2:56:41 [ERROR] Error running query, slave SQL thread aborted. Fix the problem, and restart the slave SQL thread with "SLAVE START". We stopped at log 'mysql-bin.007762' position 645226271; GTID position '77-3-552,0-3-454870353,3-3-2,2-3-469046,17-3-180,1-3-7'
          150604 2:56:41 [Note] Error reading relay log event: slave SQL thread was killed}}

          It is always the same table that is involved in this issue, and in all cases we analyzed in deep, it was the same value for the unique key to be written.

          The replication is based on one master and one slave, no restrictions on the data to be transferred, and no direct write requests to the slave. Once a day the slave is stopped for a short moment to synchronize the file system and take a backup on LVM level. At the moment we see no connection between this backup and the error (the point of time it happens seems to be independent)

          The table that is involved is defined as follows:

          {{
          CREATE TABLE `queueentry` (
          `OID` INT(11) NOT NULL AUTO_INCREMENT,
          `identry` VARCHAR(100) NOT NULL,
          `entrydate` DATETIME NULL DEFAULT NULL,
          `removedate` DATETIME NULL DEFAULT NULL,
          `info` VARCHAR(8096) NULL DEFAULT NULL,
          `prio` INT(11) NULL DEFAULT NULL,
          `reserved` DATETIME NULL DEFAULT NULL,
          `idqueue` INT(11) NOT NULL,
          `ObjectContext` VARCHAR(200) NULL DEFAULT NULL,
          `OriginHost` VARCHAR(200) NULL DEFAULT NULL,
          `ReservingHost` VARCHAR(200) NULL DEFAULT NULL,
          `UnreserveDate` DATETIME NULL DEFAULT NULL,
          `ReservationCounter` INT(10) UNSIGNED NOT NULL DEFAULT '0',
          `OptimisticLockField` INT(11) NULL DEFAULT NULL,
          PRIMARY KEY (`OID`),
          UNIQUE INDEX `identry_idqueue` (`identry`, `idqueue`),
          INDEX `idqueue` (`idqueue`),
          INDEX `entrydate_prio_reserved_idqueue` (`idqueue`, `reserved`, `prio`, `entrydate`),
          INDEX `removedate` (`removedate`),
          INDEX `entrydate` (`entrydate`),
          INDEX `UnreserveDate_ReservationCounter` (`ReservationCounter`, `UnreserveDate`)
          )
          COLLATE='utf8_general_ci'
          ENGINE=InnoDB
          AUTO_INCREMENT=8164332;
          }}

          We analyzed the binlog of the master in those cases and found the following:
          * Just before this insert statement there are two transactions bound together for parallel replication.
          * Both of these transactions work on the same table as the insert does.
          * The second of these transactions deletes the entry with the unique key, that will be written by the statement showing the duplicate key error.

          That is the following two statements are bound together:
          {{
          insert into `queueentry`(`identry`,`idqueue`,`entrydate`,`removedate`,`info`,`prio`,`reserved`,`ObjectContext`,`OriginHost`,`ReservingHost`,`UnreserveDate`,`ReservationCounter`,`OptimisticLockField`)values('SME_PrintLabel_0',249,'2015-06-04 02:56:40.671','2015-06-04 03:11:40.656','',0,null,'semaphore','172.16.185.21',null,null,0,0)
          delete from `queueentry` where ((`OID` = 6363790) and (`OptimisticLockField` = 0))
          }}

          And then this next transaction crashes:
          {{
          insert into `queueentry`(`identry`,`idqueue`,`entrydate`,`removedate`,`info`,`prio`,`reserved`,`ObjectContext`,`OriginHost`,`ReservingHost`,`UnreserveDate`,`ReservationCounter`,`OptimisticLockField`)values('VerifyOrder_0',249,'2015-06-04 02:56:40.687','2015-06-04 03:11:40.671','',0,null,'semaphore','172.16.185.22',null,null,0,0)
          }}

          Here is the (I hope) most important part of the binlog of the master:

          {{
          #150604 2:56:40 server id 3 end_log_pos 645225814 GTID 0-3-454870353 cid=5620557227
          /*!100001 SET @@session.gtid_seq_no=454870353*//*!*/;
          BEGIN
          /*!*/;
          # at 645225814
          #150604 2:56:40 server id 3 end_log_pos 645225842 Intvar
          SET INSERT_ID=6363805/*!*/;
          # at 645225842
          #150604 2:56:40 server id 3 end_log_pos 645226244 Query thread_id=19732488 exec_time=0 error_code=0
          SET TIMESTAMP=1433379400/*!*/;
          insert into `queueentry`(`identry`,`idqueue`,`entrydate`,`removedate`,`info`,`prio`,`reserved`,`ObjectContext`,`OriginHost`,`ReservingHost`,`UnreserveDate`,`ReservationCounter`,`OptimisticLockField`)values('SME_PrintLabel_0',249,'2015-06-04 02:56:40.671','2015-06-04 03:11:40.656','',0,null,'semaphore','172.16.185.21',null,null,0,0)
          /*!*/;
          # at 645226244
          #150604 2:56:40 server id 3 end_log_pos 645226271 Xid = 5620557224
          COMMIT/*!*/;
          # at 645226271
          #150604 2:56:40 server id 3 end_log_pos 645226311 GTID 0-3-454870354 cid=5620557227
          /*!100001 SET @@session.gtid_seq_no=454870354*//*!*/;
          BEGIN
          /*!*/;
          # at 645226311
          #150604 2:56:40 server id 3 end_log_pos 645226462 Query thread_id=19732489 exec_time=0 error_code=0
          SET TIMESTAMP=1433379400/*!*/;
          delete from `queueentry` where ((`OID` = 6363790) and (`OptimisticLockField` = 0))
          /*!*/;
          # at 645226462
          #150604 2:56:40 server id 3 end_log_pos 645226489 Xid = 5620557228
          COMMIT/*!*/;
          # at 645226489
          #150604 2:56:40 server id 3 end_log_pos 645226527 GTID 0-3-454870355
          /*!100001 SET @@session.gtid_seq_no=454870355*//*!*/;
          BEGIN
          /*!*/;
          # at 645226527
          #150604 2:56:40 server id 3 end_log_pos 645226555 Intvar
          SET INSERT_ID=6363806/*!*/;
          # at 645226555
          #150604 2:56:40 server id 3 end_log_pos 645226954 Query thread_id=19732488 exec_time=0 error_code=0
          SET TIMESTAMP=1433379400/*!*/;
          insert into `queueentry`(`identry`,`idqueue`,`entrydate`,`removedate`,`info`,`prio`,`reserved`,`ObjectContext`,`OriginHost`,`ReservingHost`,`UnreserveDate`,`ReservationCounter`,`OptimisticLockField`)values('VerifyOrder_0',249,'2015-06-04 02:56:40.687','2015-06-04 03:11:40.671','',0,null,'semaphore','172.16.185.22',null,null,0,0)
          /*!*/;
          # at 645226954
          #150604 2:56:40 server id 3 end_log_pos 645226981 Xid = 5620557238
          COMMIT/*!*/;
          # at 645226981
          }}

          The following additional observations we made:
          * The replication error only happens when the slave is at least one second behind the master. So it seems to us that there is somewhere a race condition at work which only happens when the slave is working very fast on the transactions from the master.
          * When this error happens, the command "stop slave" does not work any longer. We have to kill the database process, since some slave transactions are still waiting on each other.
          * After a restart of the database server we examine the table, and we cannot see the malicious key in the table any longer (even before the restart it is not there). We can restart the replication without further actions.
          * When we restart the replication, the slave is normally a few minutes behind the master. While catching up, we normally get the same error again.
          * As far as we can see, it is always the value 'VerifyOrder_0' that is causing the error, even though other values are written to that table as often as that value. We find this interesting, since it is - in alphabetical order - the largest value that is being stored in that table at the moment. I have only a rough understanding about indexes in databases, but I wonder if this could be related to an index reorganization due to the previous insert and delete statements.
          * The error vanishes if we set binlog_commit_wait_count=0 on the master (which is our current workaround).

          The variables on our slave are set as follows:

          {{
          'Variable_name','Value'
          'aria_block_size','8192'
          'aria_checkpoint_interval','30'
          'aria_checkpoint_log_activity','1048576'
          'aria_force_start_after_recovery_failures','0'
          'aria_group_commit','none'
          'aria_group_commit_interval','0'
          'aria_log_file_size','1073741824'
          'aria_log_purge_type','immediate'
          'aria_max_sort_file_size','9223372036853727232'
          'aria_page_checksum','ON'
          'aria_pagecache_age_threshold','300'
          'aria_pagecache_buffer_size','134217728'
          'aria_pagecache_division_limit','100'
          'aria_pagecache_file_hash_size','512'
          'aria_recover','NORMAL'
          'aria_repair_threads','1'
          'aria_sort_buffer_size','268434432'
          'aria_stats_method','nulls_unequal'
          'aria_sync_log_dir','NEWFILE'
          'aria_used_for_temp_tables','ON'
          'auto_increment_increment','1'
          'auto_increment_offset','1'
          'autocommit','ON'
          'automatic_sp_privileges','ON'
          'back_log','150'
          'basedir','/usr'
          'big_tables','OFF'
          'binlog_annotate_row_events','OFF'
          'binlog_cache_size','32768'
          'binlog_checksum','NONE'
          'binlog_commit_wait_count','0'
          'binlog_commit_wait_usec','100000'
          'binlog_direct_non_transactional_updates','OFF'
          'binlog_format','MIXED'
          'binlog_optimize_thread_scheduling','ON'
          'binlog_stmt_cache_size','32768'
          'bulk_insert_buffer_size','8388608'
          'character_set_client','utf8'
          'character_set_connection','utf8'
          'character_set_database','utf8'
          'character_set_filesystem','binary'
          'character_set_results','utf8'
          'character_set_server','utf8'
          'character_set_system','utf8'
          'character_sets_dir','/usr/share/mysql/charsets/'
          'collation_connection','utf8_general_ci'
          'collation_database','utf8_general_ci'
          'collation_server','utf8_unicode_ci'
          'completion_type','NO_CHAIN'
          'concurrent_insert','AUTO'
          'connect_timeout','10'
          'datadir','/var/lib/mysql/'
          'date_format','%Y-%m-%d'
          'datetime_format','%Y-%m-%d %H:%i:%s'
          'deadlock_search_depth_long','15'
          'deadlock_search_depth_short','4'
          'deadlock_timeout_long','50000000'
          'deadlock_timeout_short','10000'
          'debug_no_thread_alarm','OFF'
          'default_master_connection',''
          'default_regex_flags',''
          'default_storage_engine','InnoDB'
          'default_week_format','0'
          'delay_key_write','ON'
          'delayed_insert_limit','100'
          'delayed_insert_timeout','300'
          'delayed_queue_size','1000'
          'div_precision_increment','4'
          'error_count','0'
          'event_scheduler','OFF'
          'expensive_subquery_limit','100'
          'expire_logs_days','5'
          'external_user',''
          'extra_max_connections','1'
          'extra_port','0'
          'flush','OFF'
          'flush_time','0'
          'foreign_key_checks','ON'
          'ft_boolean_syntax','+ -><()~*:""&|'
          'ft_max_word_len','84'
          'ft_min_word_len','4'
          'ft_query_expansion_limit','20'
          'ft_stopword_file','(built-in)'
          'general_log','OFF'
          'general_log_file','optosql5.log'
          'group_concat_max_len','1024'
          'gtid_binlog_pos','0-3-481964408,3-3-2,2-3-504931'
          'gtid_binlog_state','0-5-479038918,0-3-481964408,3-3-2,2-3-504931'
          'gtid_current_pos','77-3-552,0-3-481964408,3-3-2,2-3-504931,17-3-180,1-3-7'
          'gtid_domain_id','0'
          'gtid_ignore_duplicates','OFF'
          'gtid_seq_no','0'
          'gtid_slave_pos','77-3-552,0-3-481964408,3-3-2,2-3-504931,17-3-180,1-3-7'
          'gtid_strict_mode','OFF'
          'have_compress','YES'
          'have_crypt','YES'
          'have_dynamic_loading','YES'
          'have_geometry','YES'
          'have_openssl','YES'
          'have_profiling','YES'
          'have_query_cache','YES'
          'have_rtree_keys','YES'
          'have_ssl','DISABLED'
          'have_symlink','DISABLED'
          'histogram_size','0'
          'histogram_type','SINGLE_PREC_HB'
          'host_cache_size','128'
          'hostname','optosql5'
          'identity','0'
          'ignore_builtin_innodb','OFF'
          'ignore_db_dirs',''
          'in_transaction','0'
          'init_connect',''
          'init_file',''
          'init_slave',''
          'innodb_adaptive_flushing','ON'
          'innodb_adaptive_flushing_lwm','10.000000'
          'innodb_adaptive_hash_index','ON'
          'innodb_adaptive_hash_index_partitions','1'
          'innodb_adaptive_max_sleep_delay','150000'
          'innodb_additional_mem_pool_size','8388608'
          'innodb_api_bk_commit_interval','5'
          'innodb_api_disable_rowlock','OFF'
          'innodb_api_enable_binlog','OFF'
          'innodb_api_enable_mdl','OFF'
          'innodb_api_trx_level','0'
          'innodb_autoextend_increment','64'
          'innodb_autoinc_lock_mode','1'
          'innodb_buffer_pool_dump_at_shutdown','OFF'
          'innodb_buffer_pool_dump_now','OFF'
          'innodb_buffer_pool_filename','ib_buffer_pool'
          'innodb_buffer_pool_instances','8'
          'innodb_buffer_pool_load_abort','OFF'
          'innodb_buffer_pool_load_at_startup','OFF'
          'innodb_buffer_pool_load_now','OFF'
          'innodb_buffer_pool_populate','OFF'
          'innodb_buffer_pool_size','34359738368'
          'innodb_change_buffer_max_size','25'
          'innodb_change_buffering','all'
          'innodb_checksum_algorithm','innodb'
          'innodb_checksums','ON'
          'innodb_cleaner_lsn_age_factor','high_checkpoint'
          'innodb_cmp_per_index_enabled','OFF'
          'innodb_commit_concurrency','0'
          'innodb_compression_failure_threshold_pct','5'
          'innodb_compression_level','6'
          'innodb_compression_pad_pct_max','50'
          'innodb_concurrency_tickets','5000'
          'innodb_corrupt_table_action','assert'
          'innodb_data_file_path','ibdata1:12M:autoextend'
          'innodb_data_home_dir',''
          'innodb_disable_sort_file_cache','OFF'
          'innodb_doublewrite','ON'
          'innodb_empty_free_list_algorithm','backoff'
          'innodb_fake_changes','OFF'
          'innodb_fast_shutdown','1'
          'innodb_file_format','Antelope'
          'innodb_file_format_check','ON'
          'innodb_file_format_max','Antelope'
          'innodb_file_per_table','ON'
          'innodb_flush_log_at_timeout','1'
          'innodb_flush_log_at_trx_commit','1'
          'innodb_flush_method',''
          'innodb_flush_neighbors','1'
          'innodb_flushing_avg_loops','30'
          'innodb_force_load_corrupted','OFF'
          'innodb_force_recovery','0'
          'innodb_foreground_preflush','exponential_backoff'
          'innodb_ft_aux_table',''
          'innodb_ft_cache_size','8000000'
          'innodb_ft_enable_diag_print','OFF'
          'innodb_ft_enable_stopword','ON'
          'innodb_ft_max_token_size','84'
          'innodb_ft_min_token_size','3'
          'innodb_ft_num_word_optimize','2000'
          'innodb_ft_result_cache_limit','2000000000'
          'innodb_ft_server_stopword_table',''
          'innodb_ft_sort_pll_degree','2'
          'innodb_ft_total_cache_size','640000000'
          'innodb_ft_user_stopword_table',''
          'innodb_io_capacity','200'
          'innodb_io_capacity_max','2000'
          'innodb_kill_idle_transaction','0'
          'innodb_large_prefix','OFF'
          'innodb_lock_wait_timeout','50'
          'innodb_locking_fake_changes','ON'
          'innodb_locks_unsafe_for_binlog','OFF'
          'innodb_log_arch_dir','./'
          'innodb_log_arch_expire_sec','0'
          'innodb_log_archive','OFF'
          'innodb_log_block_size','512'
          'innodb_log_buffer_size','8388608'
          'innodb_log_checksum_algorithm','innodb'
          'innodb_log_compressed_pages','ON'
          'innodb_log_file_size','104857600'
          'innodb_log_files_in_group','2'
          'innodb_log_group_home_dir','./'
          'innodb_lru_scan_depth','1024'
          'innodb_max_bitmap_file_size','104857600'
          'innodb_max_changed_pages','1000000'
          'innodb_max_dirty_pages_pct','75.000000'
          'innodb_max_dirty_pages_pct_lwm','0.001000'
          'innodb_max_purge_lag','0'
          'innodb_max_purge_lag_delay','0'
          'innodb_mirrored_log_groups','1'
          'innodb_monitor_disable',''
          'innodb_monitor_enable',''
          'innodb_monitor_reset',''
          'innodb_monitor_reset_all',''
          'innodb_old_blocks_pct','37'
          'innodb_old_blocks_time','1000'
          'innodb_online_alter_log_max_size','134217728'
          'innodb_open_files','512'
          'innodb_optimize_fulltext_only','OFF'
          'innodb_page_size','16384'
          'innodb_print_all_deadlocks','OFF'
          'innodb_purge_batch_size','300'
          'innodb_purge_threads','1'
          'innodb_random_read_ahead','OFF'
          'innodb_read_ahead_threshold','56'
          'innodb_read_io_threads','64'
          'innodb_read_only','OFF'
          'innodb_replication_delay','0'
          'innodb_rollback_on_timeout','OFF'
          'innodb_rollback_segments','128'
          'innodb_sched_priority_cleaner','19'
          'innodb_show_locks_held','10'
          'innodb_show_verbose_locks','0'
          'innodb_simulate_comp_failures','0'
          'innodb_sort_buffer_size','1048576'
          'innodb_spin_wait_delay','6'
          'innodb_stats_auto_recalc','ON'
          'innodb_stats_method','nulls_equal'
          'innodb_stats_modified_counter','0'
          'innodb_stats_on_metadata','OFF'
          'innodb_stats_persistent','ON'
          'innodb_stats_persistent_sample_pages','20'
          'innodb_stats_sample_pages','8'
          'innodb_stats_traditional','ON'
          'innodb_stats_transient_sample_pages','8'
          'innodb_status_output','OFF'
          'innodb_status_output_locks','OFF'
          'innodb_strict_mode','OFF'
          'innodb_support_xa','ON'
          'innodb_sync_array_size','1'
          'innodb_sync_spin_loops','30'
          'innodb_table_locks','ON'
          'innodb_thread_concurrency','0'
          'innodb_thread_sleep_delay','10000'
          'innodb_track_changed_pages','OFF'
          'innodb_undo_directory','.'
          'innodb_undo_logs','128'
          'innodb_undo_tablespaces','0'
          'innodb_use_atomic_writes','OFF'
          'innodb_use_fallocate','OFF'
          'innodb_use_global_flush_log_at_trx_commit','ON'
          'innodb_use_native_aio','ON'
          'innodb_use_stacktrace','OFF'
          'innodb_use_sys_malloc','ON'
          'innodb_version','5.6.22-71.0'
          'innodb_write_io_threads','64'
          'insert_id','0'
          'interactive_timeout','28800'
          'join_buffer_size','131072'
          'join_buffer_space_limit','2097152'
          'join_cache_level','2'
          'keep_files_on_create','OFF'
          'key_buffer_size','134217728'
          'key_cache_age_threshold','300'
          'key_cache_block_size','1024'
          'key_cache_division_limit','100'
          'key_cache_file_hash_size','512'
          'key_cache_segments','0'
          'large_files_support','ON'
          'large_page_size','0'
          'large_pages','OFF'
          'last_gtid',''
          'last_insert_id','0'
          'lc_messages','en_US'
          'lc_messages_dir',''
          'lc_time_names','en_US'
          'license','GPL'
          'local_infile','ON'
          'lock_wait_timeout','31536000'
          'locked_in_memory','OFF'
          'log_bin','ON'
          'log_bin_trust_function_creators','OFF'
          'log_error','/var/log/mysqld.log'
          'log_output','FILE'
          'log_queries_not_using_indexes','OFF'
          'log_slave_updates','ON'
          'log_slow_filter','admin,filesort,filesort_on_disk,full_join,full_scan,query_cache,query_cache_miss,tmp_table,tmp_table_on_disk'
          'log_slow_rate_limit','1'
          'log_slow_verbosity',''
          'log_warnings','1'
          'long_query_time','10.000000'
          'low_priority_updates','OFF'
          'lower_case_file_system','OFF'
          'lower_case_table_names','1'
          'master_verify_checksum','OFF'
          'max_allowed_packet','33554432'
          'max_binlog_cache_size','18446744073709547520'
          'max_binlog_size','1073741824'
          'max_binlog_stmt_cache_size','18446744073709547520'
          'max_connect_errors','100'
          'max_connections','500'
          'max_delayed_threads','20'
          'max_error_count','64'
          'max_heap_table_size','67108864'
          'max_insert_delayed_threads','20'
          'max_join_size','18446744073709551615'
          'max_length_for_sort_data','1024'
          'max_long_data_size','33554432'
          'max_prepared_stmt_count','16382'
          'max_relay_log_size','1073741824'
          'max_seeks_for_key','4294967295'
          'max_sort_length','1024'
          'max_sp_recursion_depth','0'
          'max_tmp_tables','32'
          'max_user_connections','0'
          'max_write_lock_count','4294967295'
          'metadata_locks_cache_size','1024'
          'metadata_locks_hash_instances','8'
          'min_examined_row_limit','0'
          'mrr_buffer_size','262144'
          'multi_range_count','256'
          'myisam_block_size','1024'
          'myisam_data_pointer_size','6'
          'myisam_max_sort_file_size','9223372036853727232'
          'myisam_mmap_size','18446744073709551615'
          'myisam_recover_options','DEFAULT'
          'myisam_repair_threads','1'
          'myisam_sort_buffer_size','134216704'
          'myisam_stats_method','nulls_unequal'
          'myisam_use_mmap','OFF'
          'net_buffer_length','16384'
          'net_read_timeout','30'
          'net_retry_count','10'
          'net_write_timeout','60'
          'old','OFF'
          'old_alter_table','OFF'
          'old_mode',''
          'old_passwords','OFF'
          'open_files_limit','500005'
          'optimizer_prune_level','1'
          'optimizer_search_depth','62'
          'optimizer_selectivity_sampling_limit','100'
          'optimizer_switch','index_merge=on,index_merge_union=on,index_merge_sort_union=on,index_merge_intersection=on,index_merge_sort_intersection=off,engine_condition_pushdown=off,index_condition_pushdown=on,derived_merge=on,derived_with_keys=on,firstmatch=on,loosescan=on,materialization=on,in_to_exists=on,semijoin=on,partial_match_rowid_merge=on,partial_match_table_scan=on,subquery_cache=on,mrr=off,mrr_cost_based=off,mrr_sort_keys=off,outer_join_with_cache=on,semijoin_with_cache=on,join_cache_incremental=on,join_cache_hashed=on,join_cache_bka=on,optimize_join_buffer_size=off,table_elimination=on,extended_keys=on,exists_to_in=on'
          'optimizer_use_condition_selectivity','1'
          'performance_schema','ON'
          'performance_schema_accounts_size','100'
          'performance_schema_digests_size','10000'
          'performance_schema_events_stages_history_long_size','10000'
          'performance_schema_events_stages_history_size','10'
          'performance_schema_events_statements_history_long_size','10000'
          'performance_schema_events_statements_history_size','10'
          'performance_schema_events_waits_history_long_size','10000'
          'performance_schema_events_waits_history_size','10'
          'performance_schema_hosts_size','100'
          'performance_schema_max_cond_classes','80'
          'performance_schema_max_cond_instances','400900'
          'performance_schema_max_file_classes','50'
          'performance_schema_max_file_handles','32768'
          'performance_schema_max_file_instances','2154'
          'performance_schema_max_mutex_classes','200'
          'performance_schema_max_mutex_instances','605000'
          'performance_schema_max_rwlock_classes','40'
          'performance_schema_max_rwlock_instances','202800'
          'performance_schema_max_socket_classes','10'
          'performance_schema_max_socket_instances','200020'
          'performance_schema_max_stage_classes','150'
          'performance_schema_max_statement_classes','180'
          'performance_schema_max_table_handles','800'
          'performance_schema_max_table_instances','12500'
          'performance_schema_max_thread_classes','50'
          'performance_schema_max_thread_instances','200100'
          'performance_schema_session_connect_attrs_size','512'
          'performance_schema_setup_actors_size','100'
          'performance_schema_setup_objects_size','100'
          'performance_schema_users_size','100'
          'pid_file','/var/lib/mysql/optosql5.pid'
          'plugin_dir','/usr/lib64/mysql/plugin/'
          'plugin_maturity','unknown'
          'port','3306'
          'preload_buffer_size','32768'
          'profiling','OFF'
          'profiling_history_size','15'
          'progress_report_time','5'
          'protocol_version','10'
          'proxy_user',''
          'pseudo_slave_mode','OFF'
          'pseudo_thread_id','189143'
          'query_alloc_block_size','8192'
          'query_cache_limit','1048576'
          'query_cache_min_res_unit','4096'
          'query_cache_size','67108864'
          'query_cache_strip_comments','OFF'
          'query_cache_type','ON'
          'query_cache_wlock_invalidate','OFF'
          'query_prealloc_size','8192'
          'rand_seed1','0'
          'rand_seed2','0'
          'range_alloc_block_size','4096'
          'read_buffer_size','131072'
          'read_only','ON'
          'read_rnd_buffer_size','262144'
          'relay_log',''
          'relay_log_index',''
          'relay_log_info_file','relay-log.info'
          'relay_log_purge','ON'
          'relay_log_recovery','OFF'
          'relay_log_space_limit','0'
          'replicate_annotate_row_events','OFF'
          'replicate_do_db',''
          'replicate_do_table',''
          'replicate_events_marked_for_skip','replicate'
          'replicate_ignore_db',''
          'replicate_ignore_table',''
          'replicate_wild_do_table',''
          'replicate_wild_ignore_table',''
          'report_host',''
          'report_password',''
          'report_port','3306'
          'report_user',''
          'rowid_merge_buff_size','8388608'
          'rpl_recovery_rank','0'
          'secure_auth','OFF'
          'secure_file_priv',''
          'server_id','5'
          'skip_external_locking','ON'
          'skip_name_resolve','ON'
          'skip_networking','OFF'
          'skip_replication','OFF'
          'skip_show_database','OFF'
          'slave_compressed_protocol','OFF'
          'slave_ddl_exec_mode','IDEMPOTENT'
          'slave_domain_parallel_threads','4'
          'slave_exec_mode','STRICT'
          'slave_load_tmpdir','/tmp'
          'slave_max_allowed_packet','1073741824'
          'slave_net_timeout','3600'
          'slave_parallel_max_queued','131072'
          'slave_parallel_threads','12'
          'slave_skip_errors','OFF'
          'slave_sql_verify_checksum','ON'
          'slave_transaction_retries','10'
          'slave_type_conversions',''
          'slow_launch_time','2'
          'slow_query_log','ON'
          'slow_query_log_file','optosql5-slow.log'
          'socket','/var/lib/mysql/mysql.sock'
          'sort_buffer_size','2097152'
          'sql_auto_is_null','OFF'
          'sql_big_selects','ON'
          'sql_buffer_result','OFF'
          'sql_log_bin','ON'
          'sql_log_off','OFF'
          'sql_mode',''
          'sql_notes','ON'
          'sql_quote_show_create','ON'
          'sql_safe_updates','OFF'
          'sql_select_limit','18446744073709551615'
          'sql_slave_skip_counter','0'
          'sql_warnings','OFF'
          'ssl_ca',''
          'ssl_capath',''
          'ssl_cert',''
          'ssl_cipher',''
          'ssl_crl',''
          'ssl_crlpath',''
          'ssl_key',''
          'storage_engine','InnoDB'
          'stored_program_cache','256'
          'sync_binlog','0'
          'sync_frm','ON'
          'sync_master_info','0'
          'sync_relay_log','0'
          'sync_relay_log_info','0'
          'system_time_zone','CEST'
          'table_definition_cache','400'
          'table_open_cache','512'
          'thread_cache_size','4'
          'thread_concurrency','10'
          'thread_handling','one-thread-per-connection'
          'thread_pool_idle_timeout','60'
          'thread_pool_max_threads','500'
          'thread_pool_oversubscribe','3'
          'thread_pool_size','16'
          'thread_pool_stall_limit','500'
          'thread_stack','294912'
          'time_format','%H:%i:%s'
          'time_zone','SYSTEM'
          'timed_mutexes','OFF'
          'timestamp','1434011008.405971'
          'tmp_table_size','134217728'
          'tmpdir','/tmp'
          'transaction_alloc_block_size','8192'
          'transaction_prealloc_size','4096'
          'tx_isolation','REPEATABLE-READ'
          'tx_read_only','OFF'
          'unique_checks','ON'
          'updatable_views_with_limit','YES'
          'use_stat_tables','NEVER'
          'userstat','OFF'
          'version','10.0.16-MariaDB-log'
          'version_comment','MariaDB Server'
          'version_compile_machine','x86_64'
          'version_compile_os','Linux'
          'version_malloc_library','bundled jemalloc'
          'wait_timeout','28800'
          'warning_count','0'
          }}

          The error occurs about once a week during normal operation.

          If you need further information, or if you want us to test something, please let me know.

          Best regards,

          Thomas Mischke
          thomas.mischke Thomas Mischke made changes -
          Description Hello,

          we are having a random "duplicate key" problem with parallel replication. We were trying hard to reproduce it on our test servers as well, but currently it only happens on our live servers.

          The error we get is the following:

          {{150604 2:56:41 [ERROR] Slave SQL: Error 'Duplicate entry 'VerifyOrder_0-249' for key 'identry_idqueue'' on query. Default database: 'optovision'. Query: 'insert into `queueentry`(`identry`,`idqueue`,`entrydate`,`removedate`,`info`,`prio`,`reserved`,`ObjectContext`,`OriginHost`,`ReservingHost`,`UnreserveDate`,`ReservationCounter`,`OptimisticLockField`)values('VerifyOrder_0',249,'2015-06-04 02:56:40.687','2015-06-04 03:11:40.671','',0,null,'semaphore','172.16.185.22',null,null,0,0)', Gtid 0-3-454870355, Internal MariaDB error code: 1062
          150604 2:56:41 [Warning] Slave: Duplicate entry 'VerifyOrder_0-249' for key 'identry_idqueue' Error_code: 1062
          150604 2:56:41 [ERROR] Error running query, slave SQL thread aborted. Fix the problem, and restart the slave SQL thread with "SLAVE START". We stopped at log 'mysql-bin.007762' position 645226271; GTID position '77-3-552,0-3-454870353,3-3-2,2-3-469046,17-3-180,1-3-7'
          150604 2:56:41 [Note] Error reading relay log event: slave SQL thread was killed}}

          It is always the same table that is involved in this issue, and in all cases we analyzed in deep, it was the same value for the unique key to be written.

          The replication is based on one master and one slave, no restrictions on the data to be transferred, and no direct write requests to the slave. Once a day the slave is stopped for a short moment to synchronize the file system and take a backup on LVM level. At the moment we see no connection between this backup and the error (the point of time it happens seems to be independent)

          The table that is involved is defined as follows:

          {{
          CREATE TABLE `queueentry` (
          `OID` INT(11) NOT NULL AUTO_INCREMENT,
          `identry` VARCHAR(100) NOT NULL,
          `entrydate` DATETIME NULL DEFAULT NULL,
          `removedate` DATETIME NULL DEFAULT NULL,
          `info` VARCHAR(8096) NULL DEFAULT NULL,
          `prio` INT(11) NULL DEFAULT NULL,
          `reserved` DATETIME NULL DEFAULT NULL,
          `idqueue` INT(11) NOT NULL,
          `ObjectContext` VARCHAR(200) NULL DEFAULT NULL,
          `OriginHost` VARCHAR(200) NULL DEFAULT NULL,
          `ReservingHost` VARCHAR(200) NULL DEFAULT NULL,
          `UnreserveDate` DATETIME NULL DEFAULT NULL,
          `ReservationCounter` INT(10) UNSIGNED NOT NULL DEFAULT '0',
          `OptimisticLockField` INT(11) NULL DEFAULT NULL,
          PRIMARY KEY (`OID`),
          UNIQUE INDEX `identry_idqueue` (`identry`, `idqueue`),
          INDEX `idqueue` (`idqueue`),
          INDEX `entrydate_prio_reserved_idqueue` (`idqueue`, `reserved`, `prio`, `entrydate`),
          INDEX `removedate` (`removedate`),
          INDEX `entrydate` (`entrydate`),
          INDEX `UnreserveDate_ReservationCounter` (`ReservationCounter`, `UnreserveDate`)
          )
          COLLATE='utf8_general_ci'
          ENGINE=InnoDB
          AUTO_INCREMENT=8164332;
          }}

          We analyzed the binlog of the master in those cases and found the following:
          * Just before this insert statement there are two transactions bound together for parallel replication.
          * Both of these transactions work on the same table as the insert does.
          * The second of these transactions deletes the entry with the unique key, that will be written by the statement showing the duplicate key error.

          That is the following two statements are bound together:
          {{
          insert into `queueentry`(`identry`,`idqueue`,`entrydate`,`removedate`,`info`,`prio`,`reserved`,`ObjectContext`,`OriginHost`,`ReservingHost`,`UnreserveDate`,`ReservationCounter`,`OptimisticLockField`)values('SME_PrintLabel_0',249,'2015-06-04 02:56:40.671','2015-06-04 03:11:40.656','',0,null,'semaphore','172.16.185.21',null,null,0,0)
          delete from `queueentry` where ((`OID` = 6363790) and (`OptimisticLockField` = 0))
          }}

          And then this next transaction crashes:
          {{
          insert into `queueentry`(`identry`,`idqueue`,`entrydate`,`removedate`,`info`,`prio`,`reserved`,`ObjectContext`,`OriginHost`,`ReservingHost`,`UnreserveDate`,`ReservationCounter`,`OptimisticLockField`)values('VerifyOrder_0',249,'2015-06-04 02:56:40.687','2015-06-04 03:11:40.671','',0,null,'semaphore','172.16.185.22',null,null,0,0)
          }}

          Here is the (I hope) most important part of the binlog of the master:

          {{
          #150604 2:56:40 server id 3 end_log_pos 645225814 GTID 0-3-454870353 cid=5620557227
          /*!100001 SET @@session.gtid_seq_no=454870353*//*!*/;
          BEGIN
          /*!*/;
          # at 645225814
          #150604 2:56:40 server id 3 end_log_pos 645225842 Intvar
          SET INSERT_ID=6363805/*!*/;
          # at 645225842
          #150604 2:56:40 server id 3 end_log_pos 645226244 Query thread_id=19732488 exec_time=0 error_code=0
          SET TIMESTAMP=1433379400/*!*/;
          insert into `queueentry`(`identry`,`idqueue`,`entrydate`,`removedate`,`info`,`prio`,`reserved`,`ObjectContext`,`OriginHost`,`ReservingHost`,`UnreserveDate`,`ReservationCounter`,`OptimisticLockField`)values('SME_PrintLabel_0',249,'2015-06-04 02:56:40.671','2015-06-04 03:11:40.656','',0,null,'semaphore','172.16.185.21',null,null,0,0)
          /*!*/;
          # at 645226244
          #150604 2:56:40 server id 3 end_log_pos 645226271 Xid = 5620557224
          COMMIT/*!*/;
          # at 645226271
          #150604 2:56:40 server id 3 end_log_pos 645226311 GTID 0-3-454870354 cid=5620557227
          /*!100001 SET @@session.gtid_seq_no=454870354*//*!*/;
          BEGIN
          /*!*/;
          # at 645226311
          #150604 2:56:40 server id 3 end_log_pos 645226462 Query thread_id=19732489 exec_time=0 error_code=0
          SET TIMESTAMP=1433379400/*!*/;
          delete from `queueentry` where ((`OID` = 6363790) and (`OptimisticLockField` = 0))
          /*!*/;
          # at 645226462
          #150604 2:56:40 server id 3 end_log_pos 645226489 Xid = 5620557228
          COMMIT/*!*/;
          # at 645226489
          #150604 2:56:40 server id 3 end_log_pos 645226527 GTID 0-3-454870355
          /*!100001 SET @@session.gtid_seq_no=454870355*//*!*/;
          BEGIN
          /*!*/;
          # at 645226527
          #150604 2:56:40 server id 3 end_log_pos 645226555 Intvar
          SET INSERT_ID=6363806/*!*/;
          # at 645226555
          #150604 2:56:40 server id 3 end_log_pos 645226954 Query thread_id=19732488 exec_time=0 error_code=0
          SET TIMESTAMP=1433379400/*!*/;
          insert into `queueentry`(`identry`,`idqueue`,`entrydate`,`removedate`,`info`,`prio`,`reserved`,`ObjectContext`,`OriginHost`,`ReservingHost`,`UnreserveDate`,`ReservationCounter`,`OptimisticLockField`)values('VerifyOrder_0',249,'2015-06-04 02:56:40.687','2015-06-04 03:11:40.671','',0,null,'semaphore','172.16.185.22',null,null,0,0)
          /*!*/;
          # at 645226954
          #150604 2:56:40 server id 3 end_log_pos 645226981 Xid = 5620557238
          COMMIT/*!*/;
          # at 645226981
          }}

          The following additional observations we made:
          * The replication error only happens when the slave is at least one second behind the master. So it seems to us that there is somewhere a race condition at work which only happens when the slave is working very fast on the transactions from the master.
          * When this error happens, the command "stop slave" does not work any longer. We have to kill the database process, since some slave transactions are still waiting on each other.
          * After a restart of the database server we examine the table, and we cannot see the malicious key in the table any longer (even before the restart it is not there). We can restart the replication without further actions.
          * When we restart the replication, the slave is normally a few minutes behind the master. While catching up, we normally get the same error again.
          * As far as we can see, it is always the value 'VerifyOrder_0' that is causing the error, even though other values are written to that table as often as that value. We find this interesting, since it is - in alphabetical order - the largest value that is being stored in that table at the moment. I have only a rough understanding about indexes in databases, but I wonder if this could be related to an index reorganization due to the previous insert and delete statements.
          * The error vanishes if we set binlog_commit_wait_count=0 on the master (which is our current workaround).

          The variables on our slave are set as follows:

          {{
          'Variable_name','Value'
          'aria_block_size','8192'
          'aria_checkpoint_interval','30'
          'aria_checkpoint_log_activity','1048576'
          'aria_force_start_after_recovery_failures','0'
          'aria_group_commit','none'
          'aria_group_commit_interval','0'
          'aria_log_file_size','1073741824'
          'aria_log_purge_type','immediate'
          'aria_max_sort_file_size','9223372036853727232'
          'aria_page_checksum','ON'
          'aria_pagecache_age_threshold','300'
          'aria_pagecache_buffer_size','134217728'
          'aria_pagecache_division_limit','100'
          'aria_pagecache_file_hash_size','512'
          'aria_recover','NORMAL'
          'aria_repair_threads','1'
          'aria_sort_buffer_size','268434432'
          'aria_stats_method','nulls_unequal'
          'aria_sync_log_dir','NEWFILE'
          'aria_used_for_temp_tables','ON'
          'auto_increment_increment','1'
          'auto_increment_offset','1'
          'autocommit','ON'
          'automatic_sp_privileges','ON'
          'back_log','150'
          'basedir','/usr'
          'big_tables','OFF'
          'binlog_annotate_row_events','OFF'
          'binlog_cache_size','32768'
          'binlog_checksum','NONE'
          'binlog_commit_wait_count','0'
          'binlog_commit_wait_usec','100000'
          'binlog_direct_non_transactional_updates','OFF'
          'binlog_format','MIXED'
          'binlog_optimize_thread_scheduling','ON'
          'binlog_stmt_cache_size','32768'
          'bulk_insert_buffer_size','8388608'
          'character_set_client','utf8'
          'character_set_connection','utf8'
          'character_set_database','utf8'
          'character_set_filesystem','binary'
          'character_set_results','utf8'
          'character_set_server','utf8'
          'character_set_system','utf8'
          'character_sets_dir','/usr/share/mysql/charsets/'
          'collation_connection','utf8_general_ci'
          'collation_database','utf8_general_ci'
          'collation_server','utf8_unicode_ci'
          'completion_type','NO_CHAIN'
          'concurrent_insert','AUTO'
          'connect_timeout','10'
          'datadir','/var/lib/mysql/'
          'date_format','%Y-%m-%d'
          'datetime_format','%Y-%m-%d %H:%i:%s'
          'deadlock_search_depth_long','15'
          'deadlock_search_depth_short','4'
          'deadlock_timeout_long','50000000'
          'deadlock_timeout_short','10000'
          'debug_no_thread_alarm','OFF'
          'default_master_connection',''
          'default_regex_flags',''
          'default_storage_engine','InnoDB'
          'default_week_format','0'
          'delay_key_write','ON'
          'delayed_insert_limit','100'
          'delayed_insert_timeout','300'
          'delayed_queue_size','1000'
          'div_precision_increment','4'
          'error_count','0'
          'event_scheduler','OFF'
          'expensive_subquery_limit','100'
          'expire_logs_days','5'
          'external_user',''
          'extra_max_connections','1'
          'extra_port','0'
          'flush','OFF'
          'flush_time','0'
          'foreign_key_checks','ON'
          'ft_boolean_syntax','+ -><()~*:""&|'
          'ft_max_word_len','84'
          'ft_min_word_len','4'
          'ft_query_expansion_limit','20'
          'ft_stopword_file','(built-in)'
          'general_log','OFF'
          'general_log_file','optosql5.log'
          'group_concat_max_len','1024'
          'gtid_binlog_pos','0-3-481964408,3-3-2,2-3-504931'
          'gtid_binlog_state','0-5-479038918,0-3-481964408,3-3-2,2-3-504931'
          'gtid_current_pos','77-3-552,0-3-481964408,3-3-2,2-3-504931,17-3-180,1-3-7'
          'gtid_domain_id','0'
          'gtid_ignore_duplicates','OFF'
          'gtid_seq_no','0'
          'gtid_slave_pos','77-3-552,0-3-481964408,3-3-2,2-3-504931,17-3-180,1-3-7'
          'gtid_strict_mode','OFF'
          'have_compress','YES'
          'have_crypt','YES'
          'have_dynamic_loading','YES'
          'have_geometry','YES'
          'have_openssl','YES'
          'have_profiling','YES'
          'have_query_cache','YES'
          'have_rtree_keys','YES'
          'have_ssl','DISABLED'
          'have_symlink','DISABLED'
          'histogram_size','0'
          'histogram_type','SINGLE_PREC_HB'
          'host_cache_size','128'
          'hostname','optosql5'
          'identity','0'
          'ignore_builtin_innodb','OFF'
          'ignore_db_dirs',''
          'in_transaction','0'
          'init_connect',''
          'init_file',''
          'init_slave',''
          'innodb_adaptive_flushing','ON'
          'innodb_adaptive_flushing_lwm','10.000000'
          'innodb_adaptive_hash_index','ON'
          'innodb_adaptive_hash_index_partitions','1'
          'innodb_adaptive_max_sleep_delay','150000'
          'innodb_additional_mem_pool_size','8388608'
          'innodb_api_bk_commit_interval','5'
          'innodb_api_disable_rowlock','OFF'
          'innodb_api_enable_binlog','OFF'
          'innodb_api_enable_mdl','OFF'
          'innodb_api_trx_level','0'
          'innodb_autoextend_increment','64'
          'innodb_autoinc_lock_mode','1'
          'innodb_buffer_pool_dump_at_shutdown','OFF'
          'innodb_buffer_pool_dump_now','OFF'
          'innodb_buffer_pool_filename','ib_buffer_pool'
          'innodb_buffer_pool_instances','8'
          'innodb_buffer_pool_load_abort','OFF'
          'innodb_buffer_pool_load_at_startup','OFF'
          'innodb_buffer_pool_load_now','OFF'
          'innodb_buffer_pool_populate','OFF'
          'innodb_buffer_pool_size','34359738368'
          'innodb_change_buffer_max_size','25'
          'innodb_change_buffering','all'
          'innodb_checksum_algorithm','innodb'
          'innodb_checksums','ON'
          'innodb_cleaner_lsn_age_factor','high_checkpoint'
          'innodb_cmp_per_index_enabled','OFF'
          'innodb_commit_concurrency','0'
          'innodb_compression_failure_threshold_pct','5'
          'innodb_compression_level','6'
          'innodb_compression_pad_pct_max','50'
          'innodb_concurrency_tickets','5000'
          'innodb_corrupt_table_action','assert'
          'innodb_data_file_path','ibdata1:12M:autoextend'
          'innodb_data_home_dir',''
          'innodb_disable_sort_file_cache','OFF'
          'innodb_doublewrite','ON'
          'innodb_empty_free_list_algorithm','backoff'
          'innodb_fake_changes','OFF'
          'innodb_fast_shutdown','1'
          'innodb_file_format','Antelope'
          'innodb_file_format_check','ON'
          'innodb_file_format_max','Antelope'
          'innodb_file_per_table','ON'
          'innodb_flush_log_at_timeout','1'
          'innodb_flush_log_at_trx_commit','1'
          'innodb_flush_method',''
          'innodb_flush_neighbors','1'
          'innodb_flushing_avg_loops','30'
          'innodb_force_load_corrupted','OFF'
          'innodb_force_recovery','0'
          'innodb_foreground_preflush','exponential_backoff'
          'innodb_ft_aux_table',''
          'innodb_ft_cache_size','8000000'
          'innodb_ft_enable_diag_print','OFF'
          'innodb_ft_enable_stopword','ON'
          'innodb_ft_max_token_size','84'
          'innodb_ft_min_token_size','3'
          'innodb_ft_num_word_optimize','2000'
          'innodb_ft_result_cache_limit','2000000000'
          'innodb_ft_server_stopword_table',''
          'innodb_ft_sort_pll_degree','2'
          'innodb_ft_total_cache_size','640000000'
          'innodb_ft_user_stopword_table',''
          'innodb_io_capacity','200'
          'innodb_io_capacity_max','2000'
          'innodb_kill_idle_transaction','0'
          'innodb_large_prefix','OFF'
          'innodb_lock_wait_timeout','50'
          'innodb_locking_fake_changes','ON'
          'innodb_locks_unsafe_for_binlog','OFF'
          'innodb_log_arch_dir','./'
          'innodb_log_arch_expire_sec','0'
          'innodb_log_archive','OFF'
          'innodb_log_block_size','512'
          'innodb_log_buffer_size','8388608'
          'innodb_log_checksum_algorithm','innodb'
          'innodb_log_compressed_pages','ON'
          'innodb_log_file_size','104857600'
          'innodb_log_files_in_group','2'
          'innodb_log_group_home_dir','./'
          'innodb_lru_scan_depth','1024'
          'innodb_max_bitmap_file_size','104857600'
          'innodb_max_changed_pages','1000000'
          'innodb_max_dirty_pages_pct','75.000000'
          'innodb_max_dirty_pages_pct_lwm','0.001000'
          'innodb_max_purge_lag','0'
          'innodb_max_purge_lag_delay','0'
          'innodb_mirrored_log_groups','1'
          'innodb_monitor_disable',''
          'innodb_monitor_enable',''
          'innodb_monitor_reset',''
          'innodb_monitor_reset_all',''
          'innodb_old_blocks_pct','37'
          'innodb_old_blocks_time','1000'
          'innodb_online_alter_log_max_size','134217728'
          'innodb_open_files','512'
          'innodb_optimize_fulltext_only','OFF'
          'innodb_page_size','16384'
          'innodb_print_all_deadlocks','OFF'
          'innodb_purge_batch_size','300'
          'innodb_purge_threads','1'
          'innodb_random_read_ahead','OFF'
          'innodb_read_ahead_threshold','56'
          'innodb_read_io_threads','64'
          'innodb_read_only','OFF'
          'innodb_replication_delay','0'
          'innodb_rollback_on_timeout','OFF'
          'innodb_rollback_segments','128'
          'innodb_sched_priority_cleaner','19'
          'innodb_show_locks_held','10'
          'innodb_show_verbose_locks','0'
          'innodb_simulate_comp_failures','0'
          'innodb_sort_buffer_size','1048576'
          'innodb_spin_wait_delay','6'
          'innodb_stats_auto_recalc','ON'
          'innodb_stats_method','nulls_equal'
          'innodb_stats_modified_counter','0'
          'innodb_stats_on_metadata','OFF'
          'innodb_stats_persistent','ON'
          'innodb_stats_persistent_sample_pages','20'
          'innodb_stats_sample_pages','8'
          'innodb_stats_traditional','ON'
          'innodb_stats_transient_sample_pages','8'
          'innodb_status_output','OFF'
          'innodb_status_output_locks','OFF'
          'innodb_strict_mode','OFF'
          'innodb_support_xa','ON'
          'innodb_sync_array_size','1'
          'innodb_sync_spin_loops','30'
          'innodb_table_locks','ON'
          'innodb_thread_concurrency','0'
          'innodb_thread_sleep_delay','10000'
          'innodb_track_changed_pages','OFF'
          'innodb_undo_directory','.'
          'innodb_undo_logs','128'
          'innodb_undo_tablespaces','0'
          'innodb_use_atomic_writes','OFF'
          'innodb_use_fallocate','OFF'
          'innodb_use_global_flush_log_at_trx_commit','ON'
          'innodb_use_native_aio','ON'
          'innodb_use_stacktrace','OFF'
          'innodb_use_sys_malloc','ON'
          'innodb_version','5.6.22-71.0'
          'innodb_write_io_threads','64'
          'insert_id','0'
          'interactive_timeout','28800'
          'join_buffer_size','131072'
          'join_buffer_space_limit','2097152'
          'join_cache_level','2'
          'keep_files_on_create','OFF'
          'key_buffer_size','134217728'
          'key_cache_age_threshold','300'
          'key_cache_block_size','1024'
          'key_cache_division_limit','100'
          'key_cache_file_hash_size','512'
          'key_cache_segments','0'
          'large_files_support','ON'
          'large_page_size','0'
          'large_pages','OFF'
          'last_gtid',''
          'last_insert_id','0'
          'lc_messages','en_US'
          'lc_messages_dir',''
          'lc_time_names','en_US'
          'license','GPL'
          'local_infile','ON'
          'lock_wait_timeout','31536000'
          'locked_in_memory','OFF'
          'log_bin','ON'
          'log_bin_trust_function_creators','OFF'
          'log_error','/var/log/mysqld.log'
          'log_output','FILE'
          'log_queries_not_using_indexes','OFF'
          'log_slave_updates','ON'
          'log_slow_filter','admin,filesort,filesort_on_disk,full_join,full_scan,query_cache,query_cache_miss,tmp_table,tmp_table_on_disk'
          'log_slow_rate_limit','1'
          'log_slow_verbosity',''
          'log_warnings','1'
          'long_query_time','10.000000'
          'low_priority_updates','OFF'
          'lower_case_file_system','OFF'
          'lower_case_table_names','1'
          'master_verify_checksum','OFF'
          'max_allowed_packet','33554432'
          'max_binlog_cache_size','18446744073709547520'
          'max_binlog_size','1073741824'
          'max_binlog_stmt_cache_size','18446744073709547520'
          'max_connect_errors','100'
          'max_connections','500'
          'max_delayed_threads','20'
          'max_error_count','64'
          'max_heap_table_size','67108864'
          'max_insert_delayed_threads','20'
          'max_join_size','18446744073709551615'
          'max_length_for_sort_data','1024'
          'max_long_data_size','33554432'
          'max_prepared_stmt_count','16382'
          'max_relay_log_size','1073741824'
          'max_seeks_for_key','4294967295'
          'max_sort_length','1024'
          'max_sp_recursion_depth','0'
          'max_tmp_tables','32'
          'max_user_connections','0'
          'max_write_lock_count','4294967295'
          'metadata_locks_cache_size','1024'
          'metadata_locks_hash_instances','8'
          'min_examined_row_limit','0'
          'mrr_buffer_size','262144'
          'multi_range_count','256'
          'myisam_block_size','1024'
          'myisam_data_pointer_size','6'
          'myisam_max_sort_file_size','9223372036853727232'
          'myisam_mmap_size','18446744073709551615'
          'myisam_recover_options','DEFAULT'
          'myisam_repair_threads','1'
          'myisam_sort_buffer_size','134216704'
          'myisam_stats_method','nulls_unequal'
          'myisam_use_mmap','OFF'
          'net_buffer_length','16384'
          'net_read_timeout','30'
          'net_retry_count','10'
          'net_write_timeout','60'
          'old','OFF'
          'old_alter_table','OFF'
          'old_mode',''
          'old_passwords','OFF'
          'open_files_limit','500005'
          'optimizer_prune_level','1'
          'optimizer_search_depth','62'
          'optimizer_selectivity_sampling_limit','100'
          'optimizer_switch','index_merge=on,index_merge_union=on,index_merge_sort_union=on,index_merge_intersection=on,index_merge_sort_intersection=off,engine_condition_pushdown=off,index_condition_pushdown=on,derived_merge=on,derived_with_keys=on,firstmatch=on,loosescan=on,materialization=on,in_to_exists=on,semijoin=on,partial_match_rowid_merge=on,partial_match_table_scan=on,subquery_cache=on,mrr=off,mrr_cost_based=off,mrr_sort_keys=off,outer_join_with_cache=on,semijoin_with_cache=on,join_cache_incremental=on,join_cache_hashed=on,join_cache_bka=on,optimize_join_buffer_size=off,table_elimination=on,extended_keys=on,exists_to_in=on'
          'optimizer_use_condition_selectivity','1'
          'performance_schema','ON'
          'performance_schema_accounts_size','100'
          'performance_schema_digests_size','10000'
          'performance_schema_events_stages_history_long_size','10000'
          'performance_schema_events_stages_history_size','10'
          'performance_schema_events_statements_history_long_size','10000'
          'performance_schema_events_statements_history_size','10'
          'performance_schema_events_waits_history_long_size','10000'
          'performance_schema_events_waits_history_size','10'
          'performance_schema_hosts_size','100'
          'performance_schema_max_cond_classes','80'
          'performance_schema_max_cond_instances','400900'
          'performance_schema_max_file_classes','50'
          'performance_schema_max_file_handles','32768'
          'performance_schema_max_file_instances','2154'
          'performance_schema_max_mutex_classes','200'
          'performance_schema_max_mutex_instances','605000'
          'performance_schema_max_rwlock_classes','40'
          'performance_schema_max_rwlock_instances','202800'
          'performance_schema_max_socket_classes','10'
          'performance_schema_max_socket_instances','200020'
          'performance_schema_max_stage_classes','150'
          'performance_schema_max_statement_classes','180'
          'performance_schema_max_table_handles','800'
          'performance_schema_max_table_instances','12500'
          'performance_schema_max_thread_classes','50'
          'performance_schema_max_thread_instances','200100'
          'performance_schema_session_connect_attrs_size','512'
          'performance_schema_setup_actors_size','100'
          'performance_schema_setup_objects_size','100'
          'performance_schema_users_size','100'
          'pid_file','/var/lib/mysql/optosql5.pid'
          'plugin_dir','/usr/lib64/mysql/plugin/'
          'plugin_maturity','unknown'
          'port','3306'
          'preload_buffer_size','32768'
          'profiling','OFF'
          'profiling_history_size','15'
          'progress_report_time','5'
          'protocol_version','10'
          'proxy_user',''
          'pseudo_slave_mode','OFF'
          'pseudo_thread_id','189143'
          'query_alloc_block_size','8192'
          'query_cache_limit','1048576'
          'query_cache_min_res_unit','4096'
          'query_cache_size','67108864'
          'query_cache_strip_comments','OFF'
          'query_cache_type','ON'
          'query_cache_wlock_invalidate','OFF'
          'query_prealloc_size','8192'
          'rand_seed1','0'
          'rand_seed2','0'
          'range_alloc_block_size','4096'
          'read_buffer_size','131072'
          'read_only','ON'
          'read_rnd_buffer_size','262144'
          'relay_log',''
          'relay_log_index',''
          'relay_log_info_file','relay-log.info'
          'relay_log_purge','ON'
          'relay_log_recovery','OFF'
          'relay_log_space_limit','0'
          'replicate_annotate_row_events','OFF'
          'replicate_do_db',''
          'replicate_do_table',''
          'replicate_events_marked_for_skip','replicate'
          'replicate_ignore_db',''
          'replicate_ignore_table',''
          'replicate_wild_do_table',''
          'replicate_wild_ignore_table',''
          'report_host',''
          'report_password',''
          'report_port','3306'
          'report_user',''
          'rowid_merge_buff_size','8388608'
          'rpl_recovery_rank','0'
          'secure_auth','OFF'
          'secure_file_priv',''
          'server_id','5'
          'skip_external_locking','ON'
          'skip_name_resolve','ON'
          'skip_networking','OFF'
          'skip_replication','OFF'
          'skip_show_database','OFF'
          'slave_compressed_protocol','OFF'
          'slave_ddl_exec_mode','IDEMPOTENT'
          'slave_domain_parallel_threads','4'
          'slave_exec_mode','STRICT'
          'slave_load_tmpdir','/tmp'
          'slave_max_allowed_packet','1073741824'
          'slave_net_timeout','3600'
          'slave_parallel_max_queued','131072'
          'slave_parallel_threads','12'
          'slave_skip_errors','OFF'
          'slave_sql_verify_checksum','ON'
          'slave_transaction_retries','10'
          'slave_type_conversions',''
          'slow_launch_time','2'
          'slow_query_log','ON'
          'slow_query_log_file','optosql5-slow.log'
          'socket','/var/lib/mysql/mysql.sock'
          'sort_buffer_size','2097152'
          'sql_auto_is_null','OFF'
          'sql_big_selects','ON'
          'sql_buffer_result','OFF'
          'sql_log_bin','ON'
          'sql_log_off','OFF'
          'sql_mode',''
          'sql_notes','ON'
          'sql_quote_show_create','ON'
          'sql_safe_updates','OFF'
          'sql_select_limit','18446744073709551615'
          'sql_slave_skip_counter','0'
          'sql_warnings','OFF'
          'ssl_ca',''
          'ssl_capath',''
          'ssl_cert',''
          'ssl_cipher',''
          'ssl_crl',''
          'ssl_crlpath',''
          'ssl_key',''
          'storage_engine','InnoDB'
          'stored_program_cache','256'
          'sync_binlog','0'
          'sync_frm','ON'
          'sync_master_info','0'
          'sync_relay_log','0'
          'sync_relay_log_info','0'
          'system_time_zone','CEST'
          'table_definition_cache','400'
          'table_open_cache','512'
          'thread_cache_size','4'
          'thread_concurrency','10'
          'thread_handling','one-thread-per-connection'
          'thread_pool_idle_timeout','60'
          'thread_pool_max_threads','500'
          'thread_pool_oversubscribe','3'
          'thread_pool_size','16'
          'thread_pool_stall_limit','500'
          'thread_stack','294912'
          'time_format','%H:%i:%s'
          'time_zone','SYSTEM'
          'timed_mutexes','OFF'
          'timestamp','1434011008.405971'
          'tmp_table_size','134217728'
          'tmpdir','/tmp'
          'transaction_alloc_block_size','8192'
          'transaction_prealloc_size','4096'
          'tx_isolation','REPEATABLE-READ'
          'tx_read_only','OFF'
          'unique_checks','ON'
          'updatable_views_with_limit','YES'
          'use_stat_tables','NEVER'
          'userstat','OFF'
          'version','10.0.16-MariaDB-log'
          'version_comment','MariaDB Server'
          'version_compile_machine','x86_64'
          'version_compile_os','Linux'
          'version_malloc_library','bundled jemalloc'
          'wait_timeout','28800'
          'warning_count','0'
          }}

          The error occurs about once a week during normal operation.

          If you need further information, or if you want us to test something, please let me know.

          Best regards,

          Thomas Mischke
          Hello,

          we are having a random "duplicate key" problem with parallel replication. We were trying hard to reproduce it on our test servers as well, but currently it only happens on our live servers.

          The error we get is the following:

          {{150604 2:56:41 [ERROR] Slave SQL: Error 'Duplicate entry 'VerifyOrder_0-249' for key 'identry_idqueue'' on query. Default database: 'optovision'. Query: 'insert into `queueentry`(`identry`,`idqueue`,`entrydate`,`removedate`,`info`,`prio`,`reserved`,`ObjectContext`,`OriginHost`,`ReservingHost`,`UnreserveDate`,`ReservationCounter`,`OptimisticLockField`)values('VerifyOrder_0',249,'2015-06-04 02:56:40.687','2015-06-04 03:11:40.671','',0,null,'semaphore','172.16.185.22',null,null,0,0)', Gtid 0-3-454870355, Internal MariaDB error code: 1062
          150604 2:56:41 [Warning] Slave: Duplicate entry 'VerifyOrder_0-249' for key 'identry_idqueue' Error_code: 1062
          150604 2:56:41 [ERROR] Error running query, slave SQL thread aborted. Fix the problem, and restart the slave SQL thread with "SLAVE START". We stopped at log 'mysql-bin.007762' position 645226271; GTID position '77-3-552,0-3-454870353,3-3-2,2-3-469046,17-3-180,1-3-7'
          150604 2:56:41 [Note] Error reading relay log event: slave SQL thread was killed
          }}
          It is always the same table that is involved in this issue, and in all cases we analyzed in deep, it was the same value for the unique key to be written.

          The replication is based on one master and one slave, no restrictions on the data to be transferred, and no direct write requests to the slave. Once a day the slave is stopped for a short moment to synchronize the file system and take a backup on LVM level. At the moment we see no connection between this backup and the error (the point of time it happens seems to be independent)

          The table that is involved is defined as follows:

          {{
          CREATE TABLE `queueentry` (
          `OID` INT(11) NOT NULL AUTO_INCREMENT,
          `identry` VARCHAR(100) NOT NULL,
          `entrydate` DATETIME NULL DEFAULT NULL,
          `removedate` DATETIME NULL DEFAULT NULL,
          `info` VARCHAR(8096) NULL DEFAULT NULL,
          `prio` INT(11) NULL DEFAULT NULL,
          `reserved` DATETIME NULL DEFAULT NULL,
          `idqueue` INT(11) NOT NULL,
          `ObjectContext` VARCHAR(200) NULL DEFAULT NULL,
          `OriginHost` VARCHAR(200) NULL DEFAULT NULL,
          `ReservingHost` VARCHAR(200) NULL DEFAULT NULL,
          `UnreserveDate` DATETIME NULL DEFAULT NULL,
          `ReservationCounter` INT(10) UNSIGNED NOT NULL DEFAULT '0',
          `OptimisticLockField` INT(11) NULL DEFAULT NULL,
          PRIMARY KEY (`OID`),
          UNIQUE INDEX `identry_idqueue` (`identry`, `idqueue`),
          INDEX `idqueue` (`idqueue`),
          INDEX `entrydate_prio_reserved_idqueue` (`idqueue`, `reserved`, `prio`, `entrydate`),
          INDEX `removedate` (`removedate`),
          INDEX `entrydate` (`entrydate`),
          INDEX `UnreserveDate_ReservationCounter` (`ReservationCounter`, `UnreserveDate`)
          )
          COLLATE='utf8_general_ci'
          ENGINE=InnoDB
          AUTO_INCREMENT=8164332;
          }}

          We analyzed the binlog of the master in those cases and found the following:
          * Just before this insert statement there are two transactions bound together for parallel replication.
          * Both of these transactions work on the same table as the insert does.
          * The second of these transactions deletes the entry with the unique key, that will be written by the statement showing the duplicate key error.

          That is the following two statements are bound together:
          {{
          insert into `queueentry`(`identry`,`idqueue`,`entrydate`,`removedate`,`info`,`prio`,`reserved`,`ObjectContext`,`OriginHost`,`ReservingHost`,`UnreserveDate`,`ReservationCounter`,`OptimisticLockField`)values('SME_PrintLabel_0',249,'2015-06-04 02:56:40.671','2015-06-04 03:11:40.656','',0,null,'semaphore','172.16.185.21',null,null,0,0)
          delete from `queueentry` where ((`OID` = 6363790) and (`OptimisticLockField` = 0))
          }}

          And then this next transaction crashes:
          {{
          insert into `queueentry`(`identry`,`idqueue`,`entrydate`,`removedate`,`info`,`prio`,`reserved`,`ObjectContext`,`OriginHost`,`ReservingHost`,`UnreserveDate`,`ReservationCounter`,`OptimisticLockField`)values('VerifyOrder_0',249,'2015-06-04 02:56:40.687','2015-06-04 03:11:40.671','',0,null,'semaphore','172.16.185.22',null,null,0,0)
          }}

          Here is the (I hope) most important part of the binlog of the master:

          {{
          #150604 2:56:40 server id 3 end_log_pos 645225814 GTID 0-3-454870353 cid=5620557227
          /*!100001 SET @@session.gtid_seq_no=454870353*//*!*/;
          BEGIN
          /*!*/;
          # at 645225814
          #150604 2:56:40 server id 3 end_log_pos 645225842 Intvar
          SET INSERT_ID=6363805/*!*/;
          # at 645225842
          #150604 2:56:40 server id 3 end_log_pos 645226244 Query thread_id=19732488 exec_time=0 error_code=0
          SET TIMESTAMP=1433379400/*!*/;
          insert into `queueentry`(`identry`,`idqueue`,`entrydate`,`removedate`,`info`,`prio`,`reserved`,`ObjectContext`,`OriginHost`,`ReservingHost`,`UnreserveDate`,`ReservationCounter`,`OptimisticLockField`)values('SME_PrintLabel_0',249,'2015-06-04 02:56:40.671','2015-06-04 03:11:40.656','',0,null,'semaphore','172.16.185.21',null,null,0,0)
          /*!*/;
          # at 645226244
          #150604 2:56:40 server id 3 end_log_pos 645226271 Xid = 5620557224
          COMMIT/*!*/;
          # at 645226271
          #150604 2:56:40 server id 3 end_log_pos 645226311 GTID 0-3-454870354 cid=5620557227
          /*!100001 SET @@session.gtid_seq_no=454870354*//*!*/;
          BEGIN
          /*!*/;
          # at 645226311
          #150604 2:56:40 server id 3 end_log_pos 645226462 Query thread_id=19732489 exec_time=0 error_code=0
          SET TIMESTAMP=1433379400/*!*/;
          delete from `queueentry` where ((`OID` = 6363790) and (`OptimisticLockField` = 0))
          /*!*/;
          # at 645226462
          #150604 2:56:40 server id 3 end_log_pos 645226489 Xid = 5620557228
          COMMIT/*!*/;
          # at 645226489
          #150604 2:56:40 server id 3 end_log_pos 645226527 GTID 0-3-454870355
          /*!100001 SET @@session.gtid_seq_no=454870355*//*!*/;
          BEGIN
          /*!*/;
          # at 645226527
          #150604 2:56:40 server id 3 end_log_pos 645226555 Intvar
          SET INSERT_ID=6363806/*!*/;
          # at 645226555
          #150604 2:56:40 server id 3 end_log_pos 645226954 Query thread_id=19732488 exec_time=0 error_code=0
          SET TIMESTAMP=1433379400/*!*/;
          insert into `queueentry`(`identry`,`idqueue`,`entrydate`,`removedate`,`info`,`prio`,`reserved`,`ObjectContext`,`OriginHost`,`ReservingHost`,`UnreserveDate`,`ReservationCounter`,`OptimisticLockField`)values('VerifyOrder_0',249,'2015-06-04 02:56:40.687','2015-06-04 03:11:40.671','',0,null,'semaphore','172.16.185.22',null,null,0,0)
          /*!*/;
          # at 645226954
          #150604 2:56:40 server id 3 end_log_pos 645226981 Xid = 5620557238
          COMMIT/*!*/;
          # at 645226981
          }}

          The following additional observations we made:
          * The replication error only happens when the slave is at least one second behind the master. So it seems to us that there is somewhere a race condition at work which only happens when the slave is working very fast on the transactions from the master.
          * When this error happens, the command "stop slave" does not work any longer. We have to kill the database process, since some slave transactions are still waiting on each other.
          * After a restart of the database server we examine the table, and we cannot see the malicious key in the table any longer (even before the restart it is not there). We can restart the replication without further actions.
          * When we restart the replication, the slave is normally a few minutes behind the master. While catching up, we normally get the same error again.
          * As far as we can see, it is always the value 'VerifyOrder_0' that is causing the error, even though other values are written to that table as often as that value. We find this interesting, since it is - in alphabetical order - the largest value that is being stored in that table at the moment. I have only a rough understanding about indexes in databases, but I wonder if this could be related to an index reorganization due to the previous insert and delete statements.
          * The error vanishes if we set binlog_commit_wait_count=0 on the master (which is our current workaround).

          The variables on our slave are set as follows:

          {{
          'Variable_name','Value'
          'aria_block_size','8192'
          'aria_checkpoint_interval','30'
          'aria_checkpoint_log_activity','1048576'
          'aria_force_start_after_recovery_failures','0'
          'aria_group_commit','none'
          'aria_group_commit_interval','0'
          'aria_log_file_size','1073741824'
          'aria_log_purge_type','immediate'
          'aria_max_sort_file_size','9223372036853727232'
          'aria_page_checksum','ON'
          'aria_pagecache_age_threshold','300'
          'aria_pagecache_buffer_size','134217728'
          'aria_pagecache_division_limit','100'
          'aria_pagecache_file_hash_size','512'
          'aria_recover','NORMAL'
          'aria_repair_threads','1'
          'aria_sort_buffer_size','268434432'
          'aria_stats_method','nulls_unequal'
          'aria_sync_log_dir','NEWFILE'
          'aria_used_for_temp_tables','ON'
          'auto_increment_increment','1'
          'auto_increment_offset','1'
          'autocommit','ON'
          'automatic_sp_privileges','ON'
          'back_log','150'
          'basedir','/usr'
          'big_tables','OFF'
          'binlog_annotate_row_events','OFF'
          'binlog_cache_size','32768'
          'binlog_checksum','NONE'
          'binlog_commit_wait_count','0'
          'binlog_commit_wait_usec','100000'
          'binlog_direct_non_transactional_updates','OFF'
          'binlog_format','MIXED'
          'binlog_optimize_thread_scheduling','ON'
          'binlog_stmt_cache_size','32768'
          'bulk_insert_buffer_size','8388608'
          'character_set_client','utf8'
          'character_set_connection','utf8'
          'character_set_database','utf8'
          'character_set_filesystem','binary'
          'character_set_results','utf8'
          'character_set_server','utf8'
          'character_set_system','utf8'
          'character_sets_dir','/usr/share/mysql/charsets/'
          'collation_connection','utf8_general_ci'
          'collation_database','utf8_general_ci'
          'collation_server','utf8_unicode_ci'
          'completion_type','NO_CHAIN'
          'concurrent_insert','AUTO'
          'connect_timeout','10'
          'datadir','/var/lib/mysql/'
          'date_format','%Y-%m-%d'
          'datetime_format','%Y-%m-%d %H:%i:%s'
          'deadlock_search_depth_long','15'
          'deadlock_search_depth_short','4'
          'deadlock_timeout_long','50000000'
          'deadlock_timeout_short','10000'
          'debug_no_thread_alarm','OFF'
          'default_master_connection',''
          'default_regex_flags',''
          'default_storage_engine','InnoDB'
          'default_week_format','0'
          'delay_key_write','ON'
          'delayed_insert_limit','100'
          'delayed_insert_timeout','300'
          'delayed_queue_size','1000'
          'div_precision_increment','4'
          'error_count','0'
          'event_scheduler','OFF'
          'expensive_subquery_limit','100'
          'expire_logs_days','5'
          'external_user',''
          'extra_max_connections','1'
          'extra_port','0'
          'flush','OFF'
          'flush_time','0'
          'foreign_key_checks','ON'
          'ft_boolean_syntax','+ -><()~*:""&|'
          'ft_max_word_len','84'
          'ft_min_word_len','4'
          'ft_query_expansion_limit','20'
          'ft_stopword_file','(built-in)'
          'general_log','OFF'
          'general_log_file','optosql5.log'
          'group_concat_max_len','1024'
          'gtid_binlog_pos','0-3-481964408,3-3-2,2-3-504931'
          'gtid_binlog_state','0-5-479038918,0-3-481964408,3-3-2,2-3-504931'
          'gtid_current_pos','77-3-552,0-3-481964408,3-3-2,2-3-504931,17-3-180,1-3-7'
          'gtid_domain_id','0'
          'gtid_ignore_duplicates','OFF'
          'gtid_seq_no','0'
          'gtid_slave_pos','77-3-552,0-3-481964408,3-3-2,2-3-504931,17-3-180,1-3-7'
          'gtid_strict_mode','OFF'
          'have_compress','YES'
          'have_crypt','YES'
          'have_dynamic_loading','YES'
          'have_geometry','YES'
          'have_openssl','YES'
          'have_profiling','YES'
          'have_query_cache','YES'
          'have_rtree_keys','YES'
          'have_ssl','DISABLED'
          'have_symlink','DISABLED'
          'histogram_size','0'
          'histogram_type','SINGLE_PREC_HB'
          'host_cache_size','128'
          'hostname','optosql5'
          'identity','0'
          'ignore_builtin_innodb','OFF'
          'ignore_db_dirs',''
          'in_transaction','0'
          'init_connect',''
          'init_file',''
          'init_slave',''
          'innodb_adaptive_flushing','ON'
          'innodb_adaptive_flushing_lwm','10.000000'
          'innodb_adaptive_hash_index','ON'
          'innodb_adaptive_hash_index_partitions','1'
          'innodb_adaptive_max_sleep_delay','150000'
          'innodb_additional_mem_pool_size','8388608'
          'innodb_api_bk_commit_interval','5'
          'innodb_api_disable_rowlock','OFF'
          'innodb_api_enable_binlog','OFF'
          'innodb_api_enable_mdl','OFF'
          'innodb_api_trx_level','0'
          'innodb_autoextend_increment','64'
          'innodb_autoinc_lock_mode','1'
          'innodb_buffer_pool_dump_at_shutdown','OFF'
          'innodb_buffer_pool_dump_now','OFF'
          'innodb_buffer_pool_filename','ib_buffer_pool'
          'innodb_buffer_pool_instances','8'
          'innodb_buffer_pool_load_abort','OFF'
          'innodb_buffer_pool_load_at_startup','OFF'
          'innodb_buffer_pool_load_now','OFF'
          'innodb_buffer_pool_populate','OFF'
          'innodb_buffer_pool_size','34359738368'
          'innodb_change_buffer_max_size','25'
          'innodb_change_buffering','all'
          'innodb_checksum_algorithm','innodb'
          'innodb_checksums','ON'
          'innodb_cleaner_lsn_age_factor','high_checkpoint'
          'innodb_cmp_per_index_enabled','OFF'
          'innodb_commit_concurrency','0'
          'innodb_compression_failure_threshold_pct','5'
          'innodb_compression_level','6'
          'innodb_compression_pad_pct_max','50'
          'innodb_concurrency_tickets','5000'
          'innodb_corrupt_table_action','assert'
          'innodb_data_file_path','ibdata1:12M:autoextend'
          'innodb_data_home_dir',''
          'innodb_disable_sort_file_cache','OFF'
          'innodb_doublewrite','ON'
          'innodb_empty_free_list_algorithm','backoff'
          'innodb_fake_changes','OFF'
          'innodb_fast_shutdown','1'
          'innodb_file_format','Antelope'
          'innodb_file_format_check','ON'
          'innodb_file_format_max','Antelope'
          'innodb_file_per_table','ON'
          'innodb_flush_log_at_timeout','1'
          'innodb_flush_log_at_trx_commit','1'
          'innodb_flush_method',''
          'innodb_flush_neighbors','1'
          'innodb_flushing_avg_loops','30'
          'innodb_force_load_corrupted','OFF'
          'innodb_force_recovery','0'
          'innodb_foreground_preflush','exponential_backoff'
          'innodb_ft_aux_table',''
          'innodb_ft_cache_size','8000000'
          'innodb_ft_enable_diag_print','OFF'
          'innodb_ft_enable_stopword','ON'
          'innodb_ft_max_token_size','84'
          'innodb_ft_min_token_size','3'
          'innodb_ft_num_word_optimize','2000'
          'innodb_ft_result_cache_limit','2000000000'
          'innodb_ft_server_stopword_table',''
          'innodb_ft_sort_pll_degree','2'
          'innodb_ft_total_cache_size','640000000'
          'innodb_ft_user_stopword_table',''
          'innodb_io_capacity','200'
          'innodb_io_capacity_max','2000'
          'innodb_kill_idle_transaction','0'
          'innodb_large_prefix','OFF'
          'innodb_lock_wait_timeout','50'
          'innodb_locking_fake_changes','ON'
          'innodb_locks_unsafe_for_binlog','OFF'
          'innodb_log_arch_dir','./'
          'innodb_log_arch_expire_sec','0'
          'innodb_log_archive','OFF'
          'innodb_log_block_size','512'
          'innodb_log_buffer_size','8388608'
          'innodb_log_checksum_algorithm','innodb'
          'innodb_log_compressed_pages','ON'
          'innodb_log_file_size','104857600'
          'innodb_log_files_in_group','2'
          'innodb_log_group_home_dir','./'
          'innodb_lru_scan_depth','1024'
          'innodb_max_bitmap_file_size','104857600'
          'innodb_max_changed_pages','1000000'
          'innodb_max_dirty_pages_pct','75.000000'
          'innodb_max_dirty_pages_pct_lwm','0.001000'
          'innodb_max_purge_lag','0'
          'innodb_max_purge_lag_delay','0'
          'innodb_mirrored_log_groups','1'
          'innodb_monitor_disable',''
          'innodb_monitor_enable',''
          'innodb_monitor_reset',''
          'innodb_monitor_reset_all',''
          'innodb_old_blocks_pct','37'
          'innodb_old_blocks_time','1000'
          'innodb_online_alter_log_max_size','134217728'
          'innodb_open_files','512'
          'innodb_optimize_fulltext_only','OFF'
          'innodb_page_size','16384'
          'innodb_print_all_deadlocks','OFF'
          'innodb_purge_batch_size','300'
          'innodb_purge_threads','1'
          'innodb_random_read_ahead','OFF'
          'innodb_read_ahead_threshold','56'
          'innodb_read_io_threads','64'
          'innodb_read_only','OFF'
          'innodb_replication_delay','0'
          'innodb_rollback_on_timeout','OFF'
          'innodb_rollback_segments','128'
          'innodb_sched_priority_cleaner','19'
          'innodb_show_locks_held','10'
          'innodb_show_verbose_locks','0'
          'innodb_simulate_comp_failures','0'
          'innodb_sort_buffer_size','1048576'
          'innodb_spin_wait_delay','6'
          'innodb_stats_auto_recalc','ON'
          'innodb_stats_method','nulls_equal'
          'innodb_stats_modified_counter','0'
          'innodb_stats_on_metadata','OFF'
          'innodb_stats_persistent','ON'
          'innodb_stats_persistent_sample_pages','20'
          'innodb_stats_sample_pages','8'
          'innodb_stats_traditional','ON'
          'innodb_stats_transient_sample_pages','8'
          'innodb_status_output','OFF'
          'innodb_status_output_locks','OFF'
          'innodb_strict_mode','OFF'
          'innodb_support_xa','ON'
          'innodb_sync_array_size','1'
          'innodb_sync_spin_loops','30'
          'innodb_table_locks','ON'
          'innodb_thread_concurrency','0'
          'innodb_thread_sleep_delay','10000'
          'innodb_track_changed_pages','OFF'
          'innodb_undo_directory','.'
          'innodb_undo_logs','128'
          'innodb_undo_tablespaces','0'
          'innodb_use_atomic_writes','OFF'
          'innodb_use_fallocate','OFF'
          'innodb_use_global_flush_log_at_trx_commit','ON'
          'innodb_use_native_aio','ON'
          'innodb_use_stacktrace','OFF'
          'innodb_use_sys_malloc','ON'
          'innodb_version','5.6.22-71.0'
          'innodb_write_io_threads','64'
          'insert_id','0'
          'interactive_timeout','28800'
          'join_buffer_size','131072'
          'join_buffer_space_limit','2097152'
          'join_cache_level','2'
          'keep_files_on_create','OFF'
          'key_buffer_size','134217728'
          'key_cache_age_threshold','300'
          'key_cache_block_size','1024'
          'key_cache_division_limit','100'
          'key_cache_file_hash_size','512'
          'key_cache_segments','0'
          'large_files_support','ON'
          'large_page_size','0'
          'large_pages','OFF'
          'last_gtid',''
          'last_insert_id','0'
          'lc_messages','en_US'
          'lc_messages_dir',''
          'lc_time_names','en_US'
          'license','GPL'
          'local_infile','ON'
          'lock_wait_timeout','31536000'
          'locked_in_memory','OFF'
          'log_bin','ON'
          'log_bin_trust_function_creators','OFF'
          'log_error','/var/log/mysqld.log'
          'log_output','FILE'
          'log_queries_not_using_indexes','OFF'
          'log_slave_updates','ON'
          'log_slow_filter','admin,filesort,filesort_on_disk,full_join,full_scan,query_cache,query_cache_miss,tmp_table,tmp_table_on_disk'
          'log_slow_rate_limit','1'
          'log_slow_verbosity',''
          'log_warnings','1'
          'long_query_time','10.000000'
          'low_priority_updates','OFF'
          'lower_case_file_system','OFF'
          'lower_case_table_names','1'
          'master_verify_checksum','OFF'
          'max_allowed_packet','33554432'
          'max_binlog_cache_size','18446744073709547520'
          'max_binlog_size','1073741824'
          'max_binlog_stmt_cache_size','18446744073709547520'
          'max_connect_errors','100'
          'max_connections','500'
          'max_delayed_threads','20'
          'max_error_count','64'
          'max_heap_table_size','67108864'
          'max_insert_delayed_threads','20'
          'max_join_size','18446744073709551615'
          'max_length_for_sort_data','1024'
          'max_long_data_size','33554432'
          'max_prepared_stmt_count','16382'
          'max_relay_log_size','1073741824'
          'max_seeks_for_key','4294967295'
          'max_sort_length','1024'
          'max_sp_recursion_depth','0'
          'max_tmp_tables','32'
          'max_user_connections','0'
          'max_write_lock_count','4294967295'
          'metadata_locks_cache_size','1024'
          'metadata_locks_hash_instances','8'
          'min_examined_row_limit','0'
          'mrr_buffer_size','262144'
          'multi_range_count','256'
          'myisam_block_size','1024'
          'myisam_data_pointer_size','6'
          'myisam_max_sort_file_size','9223372036853727232'
          'myisam_mmap_size','18446744073709551615'
          'myisam_recover_options','DEFAULT'
          'myisam_repair_threads','1'
          'myisam_sort_buffer_size','134216704'
          'myisam_stats_method','nulls_unequal'
          'myisam_use_mmap','OFF'
          'net_buffer_length','16384'
          'net_read_timeout','30'
          'net_retry_count','10'
          'net_write_timeout','60'
          'old','OFF'
          'old_alter_table','OFF'
          'old_mode',''
          'old_passwords','OFF'
          'open_files_limit','500005'
          'optimizer_prune_level','1'
          'optimizer_search_depth','62'
          'optimizer_selectivity_sampling_limit','100'
          'optimizer_switch','index_merge=on,index_merge_union=on,index_merge_sort_union=on,index_merge_intersection=on,index_merge_sort_intersection=off,engine_condition_pushdown=off,index_condition_pushdown=on,derived_merge=on,derived_with_keys=on,firstmatch=on,loosescan=on,materialization=on,in_to_exists=on,semijoin=on,partial_match_rowid_merge=on,partial_match_table_scan=on,subquery_cache=on,mrr=off,mrr_cost_based=off,mrr_sort_keys=off,outer_join_with_cache=on,semijoin_with_cache=on,join_cache_incremental=on,join_cache_hashed=on,join_cache_bka=on,optimize_join_buffer_size=off,table_elimination=on,extended_keys=on,exists_to_in=on'
          'optimizer_use_condition_selectivity','1'
          'performance_schema','ON'
          'performance_schema_accounts_size','100'
          'performance_schema_digests_size','10000'
          'performance_schema_events_stages_history_long_size','10000'
          'performance_schema_events_stages_history_size','10'
          'performance_schema_events_statements_history_long_size','10000'
          'performance_schema_events_statements_history_size','10'
          'performance_schema_events_waits_history_long_size','10000'
          'performance_schema_events_waits_history_size','10'
          'performance_schema_hosts_size','100'
          'performance_schema_max_cond_classes','80'
          'performance_schema_max_cond_instances','400900'
          'performance_schema_max_file_classes','50'
          'performance_schema_max_file_handles','32768'
          'performance_schema_max_file_instances','2154'
          'performance_schema_max_mutex_classes','200'
          'performance_schema_max_mutex_instances','605000'
          'performance_schema_max_rwlock_classes','40'
          'performance_schema_max_rwlock_instances','202800'
          'performance_schema_max_socket_classes','10'
          'performance_schema_max_socket_instances','200020'
          'performance_schema_max_stage_classes','150'
          'performance_schema_max_statement_classes','180'
          'performance_schema_max_table_handles','800'
          'performance_schema_max_table_instances','12500'
          'performance_schema_max_thread_classes','50'
          'performance_schema_max_thread_instances','200100'
          'performance_schema_session_connect_attrs_size','512'
          'performance_schema_setup_actors_size','100'
          'performance_schema_setup_objects_size','100'
          'performance_schema_users_size','100'
          'pid_file','/var/lib/mysql/optosql5.pid'
          'plugin_dir','/usr/lib64/mysql/plugin/'
          'plugin_maturity','unknown'
          'port','3306'
          'preload_buffer_size','32768'
          'profiling','OFF'
          'profiling_history_size','15'
          'progress_report_time','5'
          'protocol_version','10'
          'proxy_user',''
          'pseudo_slave_mode','OFF'
          'pseudo_thread_id','189143'
          'query_alloc_block_size','8192'
          'query_cache_limit','1048576'
          'query_cache_min_res_unit','4096'
          'query_cache_size','67108864'
          'query_cache_strip_comments','OFF'
          'query_cache_type','ON'
          'query_cache_wlock_invalidate','OFF'
          'query_prealloc_size','8192'
          'rand_seed1','0'
          'rand_seed2','0'
          'range_alloc_block_size','4096'
          'read_buffer_size','131072'
          'read_only','ON'
          'read_rnd_buffer_size','262144'
          'relay_log',''
          'relay_log_index',''
          'relay_log_info_file','relay-log.info'
          'relay_log_purge','ON'
          'relay_log_recovery','OFF'
          'relay_log_space_limit','0'
          'replicate_annotate_row_events','OFF'
          'replicate_do_db',''
          'replicate_do_table',''
          'replicate_events_marked_for_skip','replicate'
          'replicate_ignore_db',''
          'replicate_ignore_table',''
          'replicate_wild_do_table',''
          'replicate_wild_ignore_table',''
          'report_host',''
          'report_password',''
          'report_port','3306'
          'report_user',''
          'rowid_merge_buff_size','8388608'
          'rpl_recovery_rank','0'
          'secure_auth','OFF'
          'secure_file_priv',''
          'server_id','5'
          'skip_external_locking','ON'
          'skip_name_resolve','ON'
          'skip_networking','OFF'
          'skip_replication','OFF'
          'skip_show_database','OFF'
          'slave_compressed_protocol','OFF'
          'slave_ddl_exec_mode','IDEMPOTENT'
          'slave_domain_parallel_threads','4'
          'slave_exec_mode','STRICT'
          'slave_load_tmpdir','/tmp'
          'slave_max_allowed_packet','1073741824'
          'slave_net_timeout','3600'
          'slave_parallel_max_queued','131072'
          'slave_parallel_threads','12'
          'slave_skip_errors','OFF'
          'slave_sql_verify_checksum','ON'
          'slave_transaction_retries','10'
          'slave_type_conversions',''
          'slow_launch_time','2'
          'slow_query_log','ON'
          'slow_query_log_file','optosql5-slow.log'
          'socket','/var/lib/mysql/mysql.sock'
          'sort_buffer_size','2097152'
          'sql_auto_is_null','OFF'
          'sql_big_selects','ON'
          'sql_buffer_result','OFF'
          'sql_log_bin','ON'
          'sql_log_off','OFF'
          'sql_mode',''
          'sql_notes','ON'
          'sql_quote_show_create','ON'
          'sql_safe_updates','OFF'
          'sql_select_limit','18446744073709551615'
          'sql_slave_skip_counter','0'
          'sql_warnings','OFF'
          'ssl_ca',''
          'ssl_capath',''
          'ssl_cert',''
          'ssl_cipher',''
          'ssl_crl',''
          'ssl_crlpath',''
          'ssl_key',''
          'storage_engine','InnoDB'
          'stored_program_cache','256'
          'sync_binlog','0'
          'sync_frm','ON'
          'sync_master_info','0'
          'sync_relay_log','0'
          'sync_relay_log_info','0'
          'system_time_zone','CEST'
          'table_definition_cache','400'
          'table_open_cache','512'
          'thread_cache_size','4'
          'thread_concurrency','10'
          'thread_handling','one-thread-per-connection'
          'thread_pool_idle_timeout','60'
          'thread_pool_max_threads','500'
          'thread_pool_oversubscribe','3'
          'thread_pool_size','16'
          'thread_pool_stall_limit','500'
          'thread_stack','294912'
          'time_format','%H:%i:%s'
          'time_zone','SYSTEM'
          'timed_mutexes','OFF'
          'timestamp','1434011008.405971'
          'tmp_table_size','134217728'
          'tmpdir','/tmp'
          'transaction_alloc_block_size','8192'
          'transaction_prealloc_size','4096'
          'tx_isolation','REPEATABLE-READ'
          'tx_read_only','OFF'
          'unique_checks','ON'
          'updatable_views_with_limit','YES'
          'use_stat_tables','NEVER'
          'userstat','OFF'
          'version','10.0.16-MariaDB-log'
          'version_comment','MariaDB Server'
          'version_compile_machine','x86_64'
          'version_compile_os','Linux'
          'version_malloc_library','bundled jemalloc'
          'wait_timeout','28800'
          'warning_count','0'
          }}

          The error occurs about once a week during normal operation.

          If you need further information, or if you want us to test something, please let me know.

          Best regards,

          Thomas Mischke
          thomas.mischke Thomas Mischke made changes -
          Description Hello,

          we are having a random "duplicate key" problem with parallel replication. We were trying hard to reproduce it on our test servers as well, but currently it only happens on our live servers.

          The error we get is the following:

          {{150604 2:56:41 [ERROR] Slave SQL: Error 'Duplicate entry 'VerifyOrder_0-249' for key 'identry_idqueue'' on query. Default database: 'optovision'. Query: 'insert into `queueentry`(`identry`,`idqueue`,`entrydate`,`removedate`,`info`,`prio`,`reserved`,`ObjectContext`,`OriginHost`,`ReservingHost`,`UnreserveDate`,`ReservationCounter`,`OptimisticLockField`)values('VerifyOrder_0',249,'2015-06-04 02:56:40.687','2015-06-04 03:11:40.671','',0,null,'semaphore','172.16.185.22',null,null,0,0)', Gtid 0-3-454870355, Internal MariaDB error code: 1062
          150604 2:56:41 [Warning] Slave: Duplicate entry 'VerifyOrder_0-249' for key 'identry_idqueue' Error_code: 1062
          150604 2:56:41 [ERROR] Error running query, slave SQL thread aborted. Fix the problem, and restart the slave SQL thread with "SLAVE START". We stopped at log 'mysql-bin.007762' position 645226271; GTID position '77-3-552,0-3-454870353,3-3-2,2-3-469046,17-3-180,1-3-7'
          150604 2:56:41 [Note] Error reading relay log event: slave SQL thread was killed
          }}
          It is always the same table that is involved in this issue, and in all cases we analyzed in deep, it was the same value for the unique key to be written.

          The replication is based on one master and one slave, no restrictions on the data to be transferred, and no direct write requests to the slave. Once a day the slave is stopped for a short moment to synchronize the file system and take a backup on LVM level. At the moment we see no connection between this backup and the error (the point of time it happens seems to be independent)

          The table that is involved is defined as follows:

          {{
          CREATE TABLE `queueentry` (
          `OID` INT(11) NOT NULL AUTO_INCREMENT,
          `identry` VARCHAR(100) NOT NULL,
          `entrydate` DATETIME NULL DEFAULT NULL,
          `removedate` DATETIME NULL DEFAULT NULL,
          `info` VARCHAR(8096) NULL DEFAULT NULL,
          `prio` INT(11) NULL DEFAULT NULL,
          `reserved` DATETIME NULL DEFAULT NULL,
          `idqueue` INT(11) NOT NULL,
          `ObjectContext` VARCHAR(200) NULL DEFAULT NULL,
          `OriginHost` VARCHAR(200) NULL DEFAULT NULL,
          `ReservingHost` VARCHAR(200) NULL DEFAULT NULL,
          `UnreserveDate` DATETIME NULL DEFAULT NULL,
          `ReservationCounter` INT(10) UNSIGNED NOT NULL DEFAULT '0',
          `OptimisticLockField` INT(11) NULL DEFAULT NULL,
          PRIMARY KEY (`OID`),
          UNIQUE INDEX `identry_idqueue` (`identry`, `idqueue`),
          INDEX `idqueue` (`idqueue`),
          INDEX `entrydate_prio_reserved_idqueue` (`idqueue`, `reserved`, `prio`, `entrydate`),
          INDEX `removedate` (`removedate`),
          INDEX `entrydate` (`entrydate`),
          INDEX `UnreserveDate_ReservationCounter` (`ReservationCounter`, `UnreserveDate`)
          )
          COLLATE='utf8_general_ci'
          ENGINE=InnoDB
          AUTO_INCREMENT=8164332;
          }}

          We analyzed the binlog of the master in those cases and found the following:
          * Just before this insert statement there are two transactions bound together for parallel replication.
          * Both of these transactions work on the same table as the insert does.
          * The second of these transactions deletes the entry with the unique key, that will be written by the statement showing the duplicate key error.

          That is the following two statements are bound together:
          {{
          insert into `queueentry`(`identry`,`idqueue`,`entrydate`,`removedate`,`info`,`prio`,`reserved`,`ObjectContext`,`OriginHost`,`ReservingHost`,`UnreserveDate`,`ReservationCounter`,`OptimisticLockField`)values('SME_PrintLabel_0',249,'2015-06-04 02:56:40.671','2015-06-04 03:11:40.656','',0,null,'semaphore','172.16.185.21',null,null,0,0)
          delete from `queueentry` where ((`OID` = 6363790) and (`OptimisticLockField` = 0))
          }}

          And then this next transaction crashes:
          {{
          insert into `queueentry`(`identry`,`idqueue`,`entrydate`,`removedate`,`info`,`prio`,`reserved`,`ObjectContext`,`OriginHost`,`ReservingHost`,`UnreserveDate`,`ReservationCounter`,`OptimisticLockField`)values('VerifyOrder_0',249,'2015-06-04 02:56:40.687','2015-06-04 03:11:40.671','',0,null,'semaphore','172.16.185.22',null,null,0,0)
          }}

          Here is the (I hope) most important part of the binlog of the master:

          {{
          #150604 2:56:40 server id 3 end_log_pos 645225814 GTID 0-3-454870353 cid=5620557227
          /*!100001 SET @@session.gtid_seq_no=454870353*//*!*/;
          BEGIN
          /*!*/;
          # at 645225814
          #150604 2:56:40 server id 3 end_log_pos 645225842 Intvar
          SET INSERT_ID=6363805/*!*/;
          # at 645225842
          #150604 2:56:40 server id 3 end_log_pos 645226244 Query thread_id=19732488 exec_time=0 error_code=0
          SET TIMESTAMP=1433379400/*!*/;
          insert into `queueentry`(`identry`,`idqueue`,`entrydate`,`removedate`,`info`,`prio`,`reserved`,`ObjectContext`,`OriginHost`,`ReservingHost`,`UnreserveDate`,`ReservationCounter`,`OptimisticLockField`)values('SME_PrintLabel_0',249,'2015-06-04 02:56:40.671','2015-06-04 03:11:40.656','',0,null,'semaphore','172.16.185.21',null,null,0,0)
          /*!*/;
          # at 645226244
          #150604 2:56:40 server id 3 end_log_pos 645226271 Xid = 5620557224
          COMMIT/*!*/;
          # at 645226271
          #150604 2:56:40 server id 3 end_log_pos 645226311 GTID 0-3-454870354 cid=5620557227
          /*!100001 SET @@session.gtid_seq_no=454870354*//*!*/;
          BEGIN
          /*!*/;
          # at 645226311
          #150604 2:56:40 server id 3 end_log_pos 645226462 Query thread_id=19732489 exec_time=0 error_code=0
          SET TIMESTAMP=1433379400/*!*/;
          delete from `queueentry` where ((`OID` = 6363790) and (`OptimisticLockField` = 0))
          /*!*/;
          # at 645226462
          #150604 2:56:40 server id 3 end_log_pos 645226489 Xid = 5620557228
          COMMIT/*!*/;
          # at 645226489
          #150604 2:56:40 server id 3 end_log_pos 645226527 GTID 0-3-454870355
          /*!100001 SET @@session.gtid_seq_no=454870355*//*!*/;
          BEGIN
          /*!*/;
          # at 645226527
          #150604 2:56:40 server id 3 end_log_pos 645226555 Intvar
          SET INSERT_ID=6363806/*!*/;
          # at 645226555
          #150604 2:56:40 server id 3 end_log_pos 645226954 Query thread_id=19732488 exec_time=0 error_code=0
          SET TIMESTAMP=1433379400/*!*/;
          insert into `queueentry`(`identry`,`idqueue`,`entrydate`,`removedate`,`info`,`prio`,`reserved`,`ObjectContext`,`OriginHost`,`ReservingHost`,`UnreserveDate`,`ReservationCounter`,`OptimisticLockField`)values('VerifyOrder_0',249,'2015-06-04 02:56:40.687','2015-06-04 03:11:40.671','',0,null,'semaphore','172.16.185.22',null,null,0,0)
          /*!*/;
          # at 645226954
          #150604 2:56:40 server id 3 end_log_pos 645226981 Xid = 5620557238
          COMMIT/*!*/;
          # at 645226981
          }}

          The following additional observations we made:
          * The replication error only happens when the slave is at least one second behind the master. So it seems to us that there is somewhere a race condition at work which only happens when the slave is working very fast on the transactions from the master.
          * When this error happens, the command "stop slave" does not work any longer. We have to kill the database process, since some slave transactions are still waiting on each other.
          * After a restart of the database server we examine the table, and we cannot see the malicious key in the table any longer (even before the restart it is not there). We can restart the replication without further actions.
          * When we restart the replication, the slave is normally a few minutes behind the master. While catching up, we normally get the same error again.
          * As far as we can see, it is always the value 'VerifyOrder_0' that is causing the error, even though other values are written to that table as often as that value. We find this interesting, since it is - in alphabetical order - the largest value that is being stored in that table at the moment. I have only a rough understanding about indexes in databases, but I wonder if this could be related to an index reorganization due to the previous insert and delete statements.
          * The error vanishes if we set binlog_commit_wait_count=0 on the master (which is our current workaround).

          The variables on our slave are set as follows:

          {{
          'Variable_name','Value'
          'aria_block_size','8192'
          'aria_checkpoint_interval','30'
          'aria_checkpoint_log_activity','1048576'
          'aria_force_start_after_recovery_failures','0'
          'aria_group_commit','none'
          'aria_group_commit_interval','0'
          'aria_log_file_size','1073741824'
          'aria_log_purge_type','immediate'
          'aria_max_sort_file_size','9223372036853727232'
          'aria_page_checksum','ON'
          'aria_pagecache_age_threshold','300'
          'aria_pagecache_buffer_size','134217728'
          'aria_pagecache_division_limit','100'
          'aria_pagecache_file_hash_size','512'
          'aria_recover','NORMAL'
          'aria_repair_threads','1'
          'aria_sort_buffer_size','268434432'
          'aria_stats_method','nulls_unequal'
          'aria_sync_log_dir','NEWFILE'
          'aria_used_for_temp_tables','ON'
          'auto_increment_increment','1'
          'auto_increment_offset','1'
          'autocommit','ON'
          'automatic_sp_privileges','ON'
          'back_log','150'
          'basedir','/usr'
          'big_tables','OFF'
          'binlog_annotate_row_events','OFF'
          'binlog_cache_size','32768'
          'binlog_checksum','NONE'
          'binlog_commit_wait_count','0'
          'binlog_commit_wait_usec','100000'
          'binlog_direct_non_transactional_updates','OFF'
          'binlog_format','MIXED'
          'binlog_optimize_thread_scheduling','ON'
          'binlog_stmt_cache_size','32768'
          'bulk_insert_buffer_size','8388608'
          'character_set_client','utf8'
          'character_set_connection','utf8'
          'character_set_database','utf8'
          'character_set_filesystem','binary'
          'character_set_results','utf8'
          'character_set_server','utf8'
          'character_set_system','utf8'
          'character_sets_dir','/usr/share/mysql/charsets/'
          'collation_connection','utf8_general_ci'
          'collation_database','utf8_general_ci'
          'collation_server','utf8_unicode_ci'
          'completion_type','NO_CHAIN'
          'concurrent_insert','AUTO'
          'connect_timeout','10'
          'datadir','/var/lib/mysql/'
          'date_format','%Y-%m-%d'
          'datetime_format','%Y-%m-%d %H:%i:%s'
          'deadlock_search_depth_long','15'
          'deadlock_search_depth_short','4'
          'deadlock_timeout_long','50000000'
          'deadlock_timeout_short','10000'
          'debug_no_thread_alarm','OFF'
          'default_master_connection',''
          'default_regex_flags',''
          'default_storage_engine','InnoDB'
          'default_week_format','0'
          'delay_key_write','ON'
          'delayed_insert_limit','100'
          'delayed_insert_timeout','300'
          'delayed_queue_size','1000'
          'div_precision_increment','4'
          'error_count','0'
          'event_scheduler','OFF'
          'expensive_subquery_limit','100'
          'expire_logs_days','5'
          'external_user',''
          'extra_max_connections','1'
          'extra_port','0'
          'flush','OFF'
          'flush_time','0'
          'foreign_key_checks','ON'
          'ft_boolean_syntax','+ -><()~*:""&|'
          'ft_max_word_len','84'
          'ft_min_word_len','4'
          'ft_query_expansion_limit','20'
          'ft_stopword_file','(built-in)'
          'general_log','OFF'
          'general_log_file','optosql5.log'
          'group_concat_max_len','1024'
          'gtid_binlog_pos','0-3-481964408,3-3-2,2-3-504931'
          'gtid_binlog_state','0-5-479038918,0-3-481964408,3-3-2,2-3-504931'
          'gtid_current_pos','77-3-552,0-3-481964408,3-3-2,2-3-504931,17-3-180,1-3-7'
          'gtid_domain_id','0'
          'gtid_ignore_duplicates','OFF'
          'gtid_seq_no','0'
          'gtid_slave_pos','77-3-552,0-3-481964408,3-3-2,2-3-504931,17-3-180,1-3-7'
          'gtid_strict_mode','OFF'
          'have_compress','YES'
          'have_crypt','YES'
          'have_dynamic_loading','YES'
          'have_geometry','YES'
          'have_openssl','YES'
          'have_profiling','YES'
          'have_query_cache','YES'
          'have_rtree_keys','YES'
          'have_ssl','DISABLED'
          'have_symlink','DISABLED'
          'histogram_size','0'
          'histogram_type','SINGLE_PREC_HB'
          'host_cache_size','128'
          'hostname','optosql5'
          'identity','0'
          'ignore_builtin_innodb','OFF'
          'ignore_db_dirs',''
          'in_transaction','0'
          'init_connect',''
          'init_file',''
          'init_slave',''
          'innodb_adaptive_flushing','ON'
          'innodb_adaptive_flushing_lwm','10.000000'
          'innodb_adaptive_hash_index','ON'
          'innodb_adaptive_hash_index_partitions','1'
          'innodb_adaptive_max_sleep_delay','150000'
          'innodb_additional_mem_pool_size','8388608'
          'innodb_api_bk_commit_interval','5'
          'innodb_api_disable_rowlock','OFF'
          'innodb_api_enable_binlog','OFF'
          'innodb_api_enable_mdl','OFF'
          'innodb_api_trx_level','0'
          'innodb_autoextend_increment','64'
          'innodb_autoinc_lock_mode','1'
          'innodb_buffer_pool_dump_at_shutdown','OFF'
          'innodb_buffer_pool_dump_now','OFF'
          'innodb_buffer_pool_filename','ib_buffer_pool'
          'innodb_buffer_pool_instances','8'
          'innodb_buffer_pool_load_abort','OFF'
          'innodb_buffer_pool_load_at_startup','OFF'
          'innodb_buffer_pool_load_now','OFF'
          'innodb_buffer_pool_populate','OFF'
          'innodb_buffer_pool_size','34359738368'
          'innodb_change_buffer_max_size','25'
          'innodb_change_buffering','all'
          'innodb_checksum_algorithm','innodb'
          'innodb_checksums','ON'
          'innodb_cleaner_lsn_age_factor','high_checkpoint'
          'innodb_cmp_per_index_enabled','OFF'
          'innodb_commit_concurrency','0'
          'innodb_compression_failure_threshold_pct','5'
          'innodb_compression_level','6'
          'innodb_compression_pad_pct_max','50'
          'innodb_concurrency_tickets','5000'
          'innodb_corrupt_table_action','assert'
          'innodb_data_file_path','ibdata1:12M:autoextend'
          'innodb_data_home_dir',''
          'innodb_disable_sort_file_cache','OFF'
          'innodb_doublewrite','ON'
          'innodb_empty_free_list_algorithm','backoff'
          'innodb_fake_changes','OFF'
          'innodb_fast_shutdown','1'
          'innodb_file_format','Antelope'
          'innodb_file_format_check','ON'
          'innodb_file_format_max','Antelope'
          'innodb_file_per_table','ON'
          'innodb_flush_log_at_timeout','1'
          'innodb_flush_log_at_trx_commit','1'
          'innodb_flush_method',''
          'innodb_flush_neighbors','1'
          'innodb_flushing_avg_loops','30'
          'innodb_force_load_corrupted','OFF'
          'innodb_force_recovery','0'
          'innodb_foreground_preflush','exponential_backoff'
          'innodb_ft_aux_table',''
          'innodb_ft_cache_size','8000000'
          'innodb_ft_enable_diag_print','OFF'
          'innodb_ft_enable_stopword','ON'
          'innodb_ft_max_token_size','84'
          'innodb_ft_min_token_size','3'
          'innodb_ft_num_word_optimize','2000'
          'innodb_ft_result_cache_limit','2000000000'
          'innodb_ft_server_stopword_table',''
          'innodb_ft_sort_pll_degree','2'
          'innodb_ft_total_cache_size','640000000'
          'innodb_ft_user_stopword_table',''
          'innodb_io_capacity','200'
          'innodb_io_capacity_max','2000'
          'innodb_kill_idle_transaction','0'
          'innodb_large_prefix','OFF'
          'innodb_lock_wait_timeout','50'
          'innodb_locking_fake_changes','ON'
          'innodb_locks_unsafe_for_binlog','OFF'
          'innodb_log_arch_dir','./'
          'innodb_log_arch_expire_sec','0'
          'innodb_log_archive','OFF'
          'innodb_log_block_size','512'
          'innodb_log_buffer_size','8388608'
          'innodb_log_checksum_algorithm','innodb'
          'innodb_log_compressed_pages','ON'
          'innodb_log_file_size','104857600'
          'innodb_log_files_in_group','2'
          'innodb_log_group_home_dir','./'
          'innodb_lru_scan_depth','1024'
          'innodb_max_bitmap_file_size','104857600'
          'innodb_max_changed_pages','1000000'
          'innodb_max_dirty_pages_pct','75.000000'
          'innodb_max_dirty_pages_pct_lwm','0.001000'
          'innodb_max_purge_lag','0'
          'innodb_max_purge_lag_delay','0'
          'innodb_mirrored_log_groups','1'
          'innodb_monitor_disable',''
          'innodb_monitor_enable',''
          'innodb_monitor_reset',''
          'innodb_monitor_reset_all',''
          'innodb_old_blocks_pct','37'
          'innodb_old_blocks_time','1000'
          'innodb_online_alter_log_max_size','134217728'
          'innodb_open_files','512'
          'innodb_optimize_fulltext_only','OFF'
          'innodb_page_size','16384'
          'innodb_print_all_deadlocks','OFF'
          'innodb_purge_batch_size','300'
          'innodb_purge_threads','1'
          'innodb_random_read_ahead','OFF'
          'innodb_read_ahead_threshold','56'
          'innodb_read_io_threads','64'
          'innodb_read_only','OFF'
          'innodb_replication_delay','0'
          'innodb_rollback_on_timeout','OFF'
          'innodb_rollback_segments','128'
          'innodb_sched_priority_cleaner','19'
          'innodb_show_locks_held','10'
          'innodb_show_verbose_locks','0'
          'innodb_simulate_comp_failures','0'
          'innodb_sort_buffer_size','1048576'
          'innodb_spin_wait_delay','6'
          'innodb_stats_auto_recalc','ON'
          'innodb_stats_method','nulls_equal'
          'innodb_stats_modified_counter','0'
          'innodb_stats_on_metadata','OFF'
          'innodb_stats_persistent','ON'
          'innodb_stats_persistent_sample_pages','20'
          'innodb_stats_sample_pages','8'
          'innodb_stats_traditional','ON'
          'innodb_stats_transient_sample_pages','8'
          'innodb_status_output','OFF'
          'innodb_status_output_locks','OFF'
          'innodb_strict_mode','OFF'
          'innodb_support_xa','ON'
          'innodb_sync_array_size','1'
          'innodb_sync_spin_loops','30'
          'innodb_table_locks','ON'
          'innodb_thread_concurrency','0'
          'innodb_thread_sleep_delay','10000'
          'innodb_track_changed_pages','OFF'
          'innodb_undo_directory','.'
          'innodb_undo_logs','128'
          'innodb_undo_tablespaces','0'
          'innodb_use_atomic_writes','OFF'
          'innodb_use_fallocate','OFF'
          'innodb_use_global_flush_log_at_trx_commit','ON'
          'innodb_use_native_aio','ON'
          'innodb_use_stacktrace','OFF'
          'innodb_use_sys_malloc','ON'
          'innodb_version','5.6.22-71.0'
          'innodb_write_io_threads','64'
          'insert_id','0'
          'interactive_timeout','28800'
          'join_buffer_size','131072'
          'join_buffer_space_limit','2097152'
          'join_cache_level','2'
          'keep_files_on_create','OFF'
          'key_buffer_size','134217728'
          'key_cache_age_threshold','300'
          'key_cache_block_size','1024'
          'key_cache_division_limit','100'
          'key_cache_file_hash_size','512'
          'key_cache_segments','0'
          'large_files_support','ON'
          'large_page_size','0'
          'large_pages','OFF'
          'last_gtid',''
          'last_insert_id','0'
          'lc_messages','en_US'
          'lc_messages_dir',''
          'lc_time_names','en_US'
          'license','GPL'
          'local_infile','ON'
          'lock_wait_timeout','31536000'
          'locked_in_memory','OFF'
          'log_bin','ON'
          'log_bin_trust_function_creators','OFF'
          'log_error','/var/log/mysqld.log'
          'log_output','FILE'
          'log_queries_not_using_indexes','OFF'
          'log_slave_updates','ON'
          'log_slow_filter','admin,filesort,filesort_on_disk,full_join,full_scan,query_cache,query_cache_miss,tmp_table,tmp_table_on_disk'
          'log_slow_rate_limit','1'
          'log_slow_verbosity',''
          'log_warnings','1'
          'long_query_time','10.000000'
          'low_priority_updates','OFF'
          'lower_case_file_system','OFF'
          'lower_case_table_names','1'
          'master_verify_checksum','OFF'
          'max_allowed_packet','33554432'
          'max_binlog_cache_size','18446744073709547520'
          'max_binlog_size','1073741824'
          'max_binlog_stmt_cache_size','18446744073709547520'
          'max_connect_errors','100'
          'max_connections','500'
          'max_delayed_threads','20'
          'max_error_count','64'
          'max_heap_table_size','67108864'
          'max_insert_delayed_threads','20'
          'max_join_size','18446744073709551615'
          'max_length_for_sort_data','1024'
          'max_long_data_size','33554432'
          'max_prepared_stmt_count','16382'
          'max_relay_log_size','1073741824'
          'max_seeks_for_key','4294967295'
          'max_sort_length','1024'
          'max_sp_recursion_depth','0'
          'max_tmp_tables','32'
          'max_user_connections','0'
          'max_write_lock_count','4294967295'
          'metadata_locks_cache_size','1024'
          'metadata_locks_hash_instances','8'
          'min_examined_row_limit','0'
          'mrr_buffer_size','262144'
          'multi_range_count','256'
          'myisam_block_size','1024'
          'myisam_data_pointer_size','6'
          'myisam_max_sort_file_size','9223372036853727232'
          'myisam_mmap_size','18446744073709551615'
          'myisam_recover_options','DEFAULT'
          'myisam_repair_threads','1'
          'myisam_sort_buffer_size','134216704'
          'myisam_stats_method','nulls_unequal'
          'myisam_use_mmap','OFF'
          'net_buffer_length','16384'
          'net_read_timeout','30'
          'net_retry_count','10'
          'net_write_timeout','60'
          'old','OFF'
          'old_alter_table','OFF'
          'old_mode',''
          'old_passwords','OFF'
          'open_files_limit','500005'
          'optimizer_prune_level','1'
          'optimizer_search_depth','62'
          'optimizer_selectivity_sampling_limit','100'
          'optimizer_switch','index_merge=on,index_merge_union=on,index_merge_sort_union=on,index_merge_intersection=on,index_merge_sort_intersection=off,engine_condition_pushdown=off,index_condition_pushdown=on,derived_merge=on,derived_with_keys=on,firstmatch=on,loosescan=on,materialization=on,in_to_exists=on,semijoin=on,partial_match_rowid_merge=on,partial_match_table_scan=on,subquery_cache=on,mrr=off,mrr_cost_based=off,mrr_sort_keys=off,outer_join_with_cache=on,semijoin_with_cache=on,join_cache_incremental=on,join_cache_hashed=on,join_cache_bka=on,optimize_join_buffer_size=off,table_elimination=on,extended_keys=on,exists_to_in=on'
          'optimizer_use_condition_selectivity','1'
          'performance_schema','ON'
          'performance_schema_accounts_size','100'
          'performance_schema_digests_size','10000'
          'performance_schema_events_stages_history_long_size','10000'
          'performance_schema_events_stages_history_size','10'
          'performance_schema_events_statements_history_long_size','10000'
          'performance_schema_events_statements_history_size','10'
          'performance_schema_events_waits_history_long_size','10000'
          'performance_schema_events_waits_history_size','10'
          'performance_schema_hosts_size','100'
          'performance_schema_max_cond_classes','80'
          'performance_schema_max_cond_instances','400900'
          'performance_schema_max_file_classes','50'
          'performance_schema_max_file_handles','32768'
          'performance_schema_max_file_instances','2154'
          'performance_schema_max_mutex_classes','200'
          'performance_schema_max_mutex_instances','605000'
          'performance_schema_max_rwlock_classes','40'
          'performance_schema_max_rwlock_instances','202800'
          'performance_schema_max_socket_classes','10'
          'performance_schema_max_socket_instances','200020'
          'performance_schema_max_stage_classes','150'
          'performance_schema_max_statement_classes','180'
          'performance_schema_max_table_handles','800'
          'performance_schema_max_table_instances','12500'
          'performance_schema_max_thread_classes','50'
          'performance_schema_max_thread_instances','200100'
          'performance_schema_session_connect_attrs_size','512'
          'performance_schema_setup_actors_size','100'
          'performance_schema_setup_objects_size','100'
          'performance_schema_users_size','100'
          'pid_file','/var/lib/mysql/optosql5.pid'
          'plugin_dir','/usr/lib64/mysql/plugin/'
          'plugin_maturity','unknown'
          'port','3306'
          'preload_buffer_size','32768'
          'profiling','OFF'
          'profiling_history_size','15'
          'progress_report_time','5'
          'protocol_version','10'
          'proxy_user',''
          'pseudo_slave_mode','OFF'
          'pseudo_thread_id','189143'
          'query_alloc_block_size','8192'
          'query_cache_limit','1048576'
          'query_cache_min_res_unit','4096'
          'query_cache_size','67108864'
          'query_cache_strip_comments','OFF'
          'query_cache_type','ON'
          'query_cache_wlock_invalidate','OFF'
          'query_prealloc_size','8192'
          'rand_seed1','0'
          'rand_seed2','0'
          'range_alloc_block_size','4096'
          'read_buffer_size','131072'
          'read_only','ON'
          'read_rnd_buffer_size','262144'
          'relay_log',''
          'relay_log_index',''
          'relay_log_info_file','relay-log.info'
          'relay_log_purge','ON'
          'relay_log_recovery','OFF'
          'relay_log_space_limit','0'
          'replicate_annotate_row_events','OFF'
          'replicate_do_db',''
          'replicate_do_table',''
          'replicate_events_marked_for_skip','replicate'
          'replicate_ignore_db',''
          'replicate_ignore_table',''
          'replicate_wild_do_table',''
          'replicate_wild_ignore_table',''
          'report_host',''
          'report_password',''
          'report_port','3306'
          'report_user',''
          'rowid_merge_buff_size','8388608'
          'rpl_recovery_rank','0'
          'secure_auth','OFF'
          'secure_file_priv',''
          'server_id','5'
          'skip_external_locking','ON'
          'skip_name_resolve','ON'
          'skip_networking','OFF'
          'skip_replication','OFF'
          'skip_show_database','OFF'
          'slave_compressed_protocol','OFF'
          'slave_ddl_exec_mode','IDEMPOTENT'
          'slave_domain_parallel_threads','4'
          'slave_exec_mode','STRICT'
          'slave_load_tmpdir','/tmp'
          'slave_max_allowed_packet','1073741824'
          'slave_net_timeout','3600'
          'slave_parallel_max_queued','131072'
          'slave_parallel_threads','12'
          'slave_skip_errors','OFF'
          'slave_sql_verify_checksum','ON'
          'slave_transaction_retries','10'
          'slave_type_conversions',''
          'slow_launch_time','2'
          'slow_query_log','ON'
          'slow_query_log_file','optosql5-slow.log'
          'socket','/var/lib/mysql/mysql.sock'
          'sort_buffer_size','2097152'
          'sql_auto_is_null','OFF'
          'sql_big_selects','ON'
          'sql_buffer_result','OFF'
          'sql_log_bin','ON'
          'sql_log_off','OFF'
          'sql_mode',''
          'sql_notes','ON'
          'sql_quote_show_create','ON'
          'sql_safe_updates','OFF'
          'sql_select_limit','18446744073709551615'
          'sql_slave_skip_counter','0'
          'sql_warnings','OFF'
          'ssl_ca',''
          'ssl_capath',''
          'ssl_cert',''
          'ssl_cipher',''
          'ssl_crl',''
          'ssl_crlpath',''
          'ssl_key',''
          'storage_engine','InnoDB'
          'stored_program_cache','256'
          'sync_binlog','0'
          'sync_frm','ON'
          'sync_master_info','0'
          'sync_relay_log','0'
          'sync_relay_log_info','0'
          'system_time_zone','CEST'
          'table_definition_cache','400'
          'table_open_cache','512'
          'thread_cache_size','4'
          'thread_concurrency','10'
          'thread_handling','one-thread-per-connection'
          'thread_pool_idle_timeout','60'
          'thread_pool_max_threads','500'
          'thread_pool_oversubscribe','3'
          'thread_pool_size','16'
          'thread_pool_stall_limit','500'
          'thread_stack','294912'
          'time_format','%H:%i:%s'
          'time_zone','SYSTEM'
          'timed_mutexes','OFF'
          'timestamp','1434011008.405971'
          'tmp_table_size','134217728'
          'tmpdir','/tmp'
          'transaction_alloc_block_size','8192'
          'transaction_prealloc_size','4096'
          'tx_isolation','REPEATABLE-READ'
          'tx_read_only','OFF'
          'unique_checks','ON'
          'updatable_views_with_limit','YES'
          'use_stat_tables','NEVER'
          'userstat','OFF'
          'version','10.0.16-MariaDB-log'
          'version_comment','MariaDB Server'
          'version_compile_machine','x86_64'
          'version_compile_os','Linux'
          'version_malloc_library','bundled jemalloc'
          'wait_timeout','28800'
          'warning_count','0'
          }}

          The error occurs about once a week during normal operation.

          If you need further information, or if you want us to test something, please let me know.

          Best regards,

          Thomas Mischke
          Hello,

          we are having a random "duplicate key" problem with parallel replication. We were trying hard to reproduce it on our test servers as well, but currently it only happens on our live servers.

          The error we get is the following:

          {quote}150604 2:56:41 [ERROR] Slave SQL: Error 'Duplicate entry 'VerifyOrder_0-249' for key 'identry_idqueue'' on query. Default database: 'optovision'. Query: 'insert into `queueentry`(`identry`,`idqueue`,`entrydate`,`removedate`,`info`,`prio`,`reserved`,`ObjectContext`,`OriginHost`,`ReservingHost`,`UnreserveDate`,`ReservationCounter`,`OptimisticLockField`)values('VerifyOrder_0',249,'2015-06-04 02:56:40.687','2015-06-04 03:11:40.671','',0,null,'semaphore','172.16.185.22',null,null,0,0)', Gtid 0-3-454870355, Internal MariaDB error code: 1062
          150604 2:56:41 [Warning] Slave: Duplicate entry 'VerifyOrder_0-249' for key 'identry_idqueue' Error_code: 1062
          150604 2:56:41 [ERROR] Error running query, slave SQL thread aborted. Fix the problem, and restart the slave SQL thread with "SLAVE START". We stopped at log 'mysql-bin.007762' position 645226271; GTID position '77-3-552,0-3-454870353,3-3-2,2-3-469046,17-3-180,1-3-7'
          150604 2:56:41 [Note] Error reading relay log event: slave SQL thread was killed{quote}

          It is always the same table that is involved in this issue, and in all cases we analyzed in deep, it was the same value for the unique key to be written.

          The replication is based on one master and one slave, no restrictions on the data to be transferred, and no direct write requests to the slave. Once a day the slave is stopped for a short moment to synchronize the file system and take a backup on LVM level. At the moment we see no connection between this backup and the error (the point of time it happens seems to be independent)

          The table that is involved is defined as follows:

          {{
          CREATE TABLE `queueentry` (
          `OID` INT(11) NOT NULL AUTO_INCREMENT,
          `identry` VARCHAR(100) NOT NULL,
          `entrydate` DATETIME NULL DEFAULT NULL,
          `removedate` DATETIME NULL DEFAULT NULL,
          `info` VARCHAR(8096) NULL DEFAULT NULL,
          `prio` INT(11) NULL DEFAULT NULL,
          `reserved` DATETIME NULL DEFAULT NULL,
          `idqueue` INT(11) NOT NULL,
          `ObjectContext` VARCHAR(200) NULL DEFAULT NULL,
          `OriginHost` VARCHAR(200) NULL DEFAULT NULL,
          `ReservingHost` VARCHAR(200) NULL DEFAULT NULL,
          `UnreserveDate` DATETIME NULL DEFAULT NULL,
          `ReservationCounter` INT(10) UNSIGNED NOT NULL DEFAULT '0',
          `OptimisticLockField` INT(11) NULL DEFAULT NULL,
          PRIMARY KEY (`OID`),
          UNIQUE INDEX `identry_idqueue` (`identry`, `idqueue`),
          INDEX `idqueue` (`idqueue`),
          INDEX `entrydate_prio_reserved_idqueue` (`idqueue`, `reserved`, `prio`, `entrydate`),
          INDEX `removedate` (`removedate`),
          INDEX `entrydate` (`entrydate`),
          INDEX `UnreserveDate_ReservationCounter` (`ReservationCounter`, `UnreserveDate`)
          )
          COLLATE='utf8_general_ci'
          ENGINE=InnoDB
          AUTO_INCREMENT=8164332;
          }}

          We analyzed the binlog of the master in those cases and found the following:
          * Just before this insert statement there are two transactions bound together for parallel replication.
          * Both of these transactions work on the same table as the insert does.
          * The second of these transactions deletes the entry with the unique key, that will be written by the statement showing the duplicate key error.

          That is the following two statements are bound together:
          {{
          insert into `queueentry`(`identry`,`idqueue`,`entrydate`,`removedate`,`info`,`prio`,`reserved`,`ObjectContext`,`OriginHost`,`ReservingHost`,`UnreserveDate`,`ReservationCounter`,`OptimisticLockField`)values('SME_PrintLabel_0',249,'2015-06-04 02:56:40.671','2015-06-04 03:11:40.656','',0,null,'semaphore','172.16.185.21',null,null,0,0)
          delete from `queueentry` where ((`OID` = 6363790) and (`OptimisticLockField` = 0))
          }}

          And then this next transaction crashes:
          {{
          insert into `queueentry`(`identry`,`idqueue`,`entrydate`,`removedate`,`info`,`prio`,`reserved`,`ObjectContext`,`OriginHost`,`ReservingHost`,`UnreserveDate`,`ReservationCounter`,`OptimisticLockField`)values('VerifyOrder_0',249,'2015-06-04 02:56:40.687','2015-06-04 03:11:40.671','',0,null,'semaphore','172.16.185.22',null,null,0,0)
          }}

          Here is the (I hope) most important part of the binlog of the master:

          {{
          #150604 2:56:40 server id 3 end_log_pos 645225814 GTID 0-3-454870353 cid=5620557227
          /*!100001 SET @@session.gtid_seq_no=454870353*//*!*/;
          BEGIN
          /*!*/;
          # at 645225814
          #150604 2:56:40 server id 3 end_log_pos 645225842 Intvar
          SET INSERT_ID=6363805/*!*/;
          # at 645225842
          #150604 2:56:40 server id 3 end_log_pos 645226244 Query thread_id=19732488 exec_time=0 error_code=0
          SET TIMESTAMP=1433379400/*!*/;
          insert into `queueentry`(`identry`,`idqueue`,`entrydate`,`removedate`,`info`,`prio`,`reserved`,`ObjectContext`,`OriginHost`,`ReservingHost`,`UnreserveDate`,`ReservationCounter`,`OptimisticLockField`)values('SME_PrintLabel_0',249,'2015-06-04 02:56:40.671','2015-06-04 03:11:40.656','',0,null,'semaphore','172.16.185.21',null,null,0,0)
          /*!*/;
          # at 645226244
          #150604 2:56:40 server id 3 end_log_pos 645226271 Xid = 5620557224
          COMMIT/*!*/;
          # at 645226271
          #150604 2:56:40 server id 3 end_log_pos 645226311 GTID 0-3-454870354 cid=5620557227
          /*!100001 SET @@session.gtid_seq_no=454870354*//*!*/;
          BEGIN
          /*!*/;
          # at 645226311
          #150604 2:56:40 server id 3 end_log_pos 645226462 Query thread_id=19732489 exec_time=0 error_code=0
          SET TIMESTAMP=1433379400/*!*/;
          delete from `queueentry` where ((`OID` = 6363790) and (`OptimisticLockField` = 0))
          /*!*/;
          # at 645226462
          #150604 2:56:40 server id 3 end_log_pos 645226489 Xid = 5620557228
          COMMIT/*!*/;
          # at 645226489
          #150604 2:56:40 server id 3 end_log_pos 645226527 GTID 0-3-454870355
          /*!100001 SET @@session.gtid_seq_no=454870355*//*!*/;
          BEGIN
          /*!*/;
          # at 645226527
          #150604 2:56:40 server id 3 end_log_pos 645226555 Intvar
          SET INSERT_ID=6363806/*!*/;
          # at 645226555
          #150604 2:56:40 server id 3 end_log_pos 645226954 Query thread_id=19732488 exec_time=0 error_code=0
          SET TIMESTAMP=1433379400/*!*/;
          insert into `queueentry`(`identry`,`idqueue`,`entrydate`,`removedate`,`info`,`prio`,`reserved`,`ObjectContext`,`OriginHost`,`ReservingHost`,`UnreserveDate`,`ReservationCounter`,`OptimisticLockField`)values('VerifyOrder_0',249,'2015-06-04 02:56:40.687','2015-06-04 03:11:40.671','',0,null,'semaphore','172.16.185.22',null,null,0,0)
          /*!*/;
          # at 645226954
          #150604 2:56:40 server id 3 end_log_pos 645226981 Xid = 5620557238
          COMMIT/*!*/;
          # at 645226981
          }}

          The following additional observations we made:
          * The replication error only happens when the slave is at least one second behind the master. So it seems to us that there is somewhere a race condition at work which only happens when the slave is working very fast on the transactions from the master.
          * When this error happens, the command "stop slave" does not work any longer. We have to kill the database process, since some slave transactions are still waiting on each other.
          * After a restart of the database server we examine the table, and we cannot see the malicious key in the table any longer (even before the restart it is not there). We can restart the replication without further actions.
          * When we restart the replication, the slave is normally a few minutes behind the master. While catching up, we normally get the same error again.
          * As far as we can see, it is always the value 'VerifyOrder_0' that is causing the error, even though other values are written to that table as often as that value. We find this interesting, since it is - in alphabetical order - the largest value that is being stored in that table at the moment. I have only a rough understanding about indexes in databases, but I wonder if this could be related to an index reorganization due to the previous insert and delete statements.
          * The error vanishes if we set binlog_commit_wait_count=0 on the master (which is our current workaround).

          The variables on our slave are set as follows:

          {{
          'Variable_name','Value'
          'aria_block_size','8192'
          'aria_checkpoint_interval','30'
          'aria_checkpoint_log_activity','1048576'
          'aria_force_start_after_recovery_failures','0'
          'aria_group_commit','none'
          'aria_group_commit_interval','0'
          'aria_log_file_size','1073741824'
          'aria_log_purge_type','immediate'
          'aria_max_sort_file_size','9223372036853727232'
          'aria_page_checksum','ON'
          'aria_pagecache_age_threshold','300'
          'aria_pagecache_buffer_size','134217728'
          'aria_pagecache_division_limit','100'
          'aria_pagecache_file_hash_size','512'
          'aria_recover','NORMAL'
          'aria_repair_threads','1'
          'aria_sort_buffer_size','268434432'
          'aria_stats_method','nulls_unequal'
          'aria_sync_log_dir','NEWFILE'
          'aria_used_for_temp_tables','ON'
          'auto_increment_increment','1'
          'auto_increment_offset','1'
          'autocommit','ON'
          'automatic_sp_privileges','ON'
          'back_log','150'
          'basedir','/usr'
          'big_tables','OFF'
          'binlog_annotate_row_events','OFF'
          'binlog_cache_size','32768'
          'binlog_checksum','NONE'
          'binlog_commit_wait_count','0'
          'binlog_commit_wait_usec','100000'
          'binlog_direct_non_transactional_updates','OFF'
          'binlog_format','MIXED'
          'binlog_optimize_thread_scheduling','ON'
          'binlog_stmt_cache_size','32768'
          'bulk_insert_buffer_size','8388608'
          'character_set_client','utf8'
          'character_set_connection','utf8'
          'character_set_database','utf8'
          'character_set_filesystem','binary'
          'character_set_results','utf8'
          'character_set_server','utf8'
          'character_set_system','utf8'
          'character_sets_dir','/usr/share/mysql/charsets/'
          'collation_connection','utf8_general_ci'
          'collation_database','utf8_general_ci'
          'collation_server','utf8_unicode_ci'
          'completion_type','NO_CHAIN'
          'concurrent_insert','AUTO'
          'connect_timeout','10'
          'datadir','/var/lib/mysql/'
          'date_format','%Y-%m-%d'
          'datetime_format','%Y-%m-%d %H:%i:%s'
          'deadlock_search_depth_long','15'
          'deadlock_search_depth_short','4'
          'deadlock_timeout_long','50000000'
          'deadlock_timeout_short','10000'
          'debug_no_thread_alarm','OFF'
          'default_master_connection',''
          'default_regex_flags',''
          'default_storage_engine','InnoDB'
          'default_week_format','0'
          'delay_key_write','ON'
          'delayed_insert_limit','100'
          'delayed_insert_timeout','300'
          'delayed_queue_size','1000'
          'div_precision_increment','4'
          'error_count','0'
          'event_scheduler','OFF'
          'expensive_subquery_limit','100'
          'expire_logs_days','5'
          'external_user',''
          'extra_max_connections','1'
          'extra_port','0'
          'flush','OFF'
          'flush_time','0'
          'foreign_key_checks','ON'
          'ft_boolean_syntax','+ -><()~*:""&|'
          'ft_max_word_len','84'
          'ft_min_word_len','4'
          'ft_query_expansion_limit','20'
          'ft_stopword_file','(built-in)'
          'general_log','OFF'
          'general_log_file','optosql5.log'
          'group_concat_max_len','1024'
          'gtid_binlog_pos','0-3-481964408,3-3-2,2-3-504931'
          'gtid_binlog_state','0-5-479038918,0-3-481964408,3-3-2,2-3-504931'
          'gtid_current_pos','77-3-552,0-3-481964408,3-3-2,2-3-504931,17-3-180,1-3-7'
          'gtid_domain_id','0'
          'gtid_ignore_duplicates','OFF'
          'gtid_seq_no','0'
          'gtid_slave_pos','77-3-552,0-3-481964408,3-3-2,2-3-504931,17-3-180,1-3-7'
          'gtid_strict_mode','OFF'
          'have_compress','YES'
          'have_crypt','YES'
          'have_dynamic_loading','YES'
          'have_geometry','YES'
          'have_openssl','YES'
          'have_profiling','YES'
          'have_query_cache','YES'
          'have_rtree_keys','YES'
          'have_ssl','DISABLED'
          'have_symlink','DISABLED'
          'histogram_size','0'
          'histogram_type','SINGLE_PREC_HB'
          'host_cache_size','128'
          'hostname','optosql5'
          'identity','0'
          'ignore_builtin_innodb','OFF'
          'ignore_db_dirs',''
          'in_transaction','0'
          'init_connect',''
          'init_file',''
          'init_slave',''
          'innodb_adaptive_flushing','ON'
          'innodb_adaptive_flushing_lwm','10.000000'
          'innodb_adaptive_hash_index','ON'
          'innodb_adaptive_hash_index_partitions','1'
          'innodb_adaptive_max_sleep_delay','150000'
          'innodb_additional_mem_pool_size','8388608'
          'innodb_api_bk_commit_interval','5'
          'innodb_api_disable_rowlock','OFF'
          'innodb_api_enable_binlog','OFF'
          'innodb_api_enable_mdl','OFF'
          'innodb_api_trx_level','0'
          'innodb_autoextend_increment','64'
          'innodb_autoinc_lock_mode','1'
          'innodb_buffer_pool_dump_at_shutdown','OFF'
          'innodb_buffer_pool_dump_now','OFF'
          'innodb_buffer_pool_filename','ib_buffer_pool'
          'innodb_buffer_pool_instances','8'
          'innodb_buffer_pool_load_abort','OFF'
          'innodb_buffer_pool_load_at_startup','OFF'
          'innodb_buffer_pool_load_now','OFF'
          'innodb_buffer_pool_populate','OFF'
          'innodb_buffer_pool_size','34359738368'
          'innodb_change_buffer_max_size','25'
          'innodb_change_buffering','all'
          'innodb_checksum_algorithm','innodb'
          'innodb_checksums','ON'
          'innodb_cleaner_lsn_age_factor','high_checkpoint'
          'innodb_cmp_per_index_enabled','OFF'
          'innodb_commit_concurrency','0'
          'innodb_compression_failure_threshold_pct','5'
          'innodb_compression_level','6'
          'innodb_compression_pad_pct_max','50'
          'innodb_concurrency_tickets','5000'
          'innodb_corrupt_table_action','assert'
          'innodb_data_file_path','ibdata1:12M:autoextend'
          'innodb_data_home_dir',''
          'innodb_disable_sort_file_cache','OFF'
          'innodb_doublewrite','ON'
          'innodb_empty_free_list_algorithm','backoff'
          'innodb_fake_changes','OFF'
          'innodb_fast_shutdown','1'
          'innodb_file_format','Antelope'
          'innodb_file_format_check','ON'
          'innodb_file_format_max','Antelope'
          'innodb_file_per_table','ON'
          'innodb_flush_log_at_timeout','1'
          'innodb_flush_log_at_trx_commit','1'
          'innodb_flush_method',''
          'innodb_flush_neighbors','1'
          'innodb_flushing_avg_loops','30'
          'innodb_force_load_corrupted','OFF'
          'innodb_force_recovery','0'
          'innodb_foreground_preflush','exponential_backoff'
          'innodb_ft_aux_table',''
          'innodb_ft_cache_size','8000000'
          'innodb_ft_enable_diag_print','OFF'
          'innodb_ft_enable_stopword','ON'
          'innodb_ft_max_token_size','84'
          'innodb_ft_min_token_size','3'
          'innodb_ft_num_word_optimize','2000'
          'innodb_ft_result_cache_limit','2000000000'
          'innodb_ft_server_stopword_table',''
          'innodb_ft_sort_pll_degree','2'
          'innodb_ft_total_cache_size','640000000'
          'innodb_ft_user_stopword_table',''
          'innodb_io_capacity','200'
          'innodb_io_capacity_max','2000'
          'innodb_kill_idle_transaction','0'
          'innodb_large_prefix','OFF'
          'innodb_lock_wait_timeout','50'
          'innodb_locking_fake_changes','ON'
          'innodb_locks_unsafe_for_binlog','OFF'
          'innodb_log_arch_dir','./'
          'innodb_log_arch_expire_sec','0'
          'innodb_log_archive','OFF'
          'innodb_log_block_size','512'
          'innodb_log_buffer_size','8388608'
          'innodb_log_checksum_algorithm','innodb'
          'innodb_log_compressed_pages','ON'
          'innodb_log_file_size','104857600'
          'innodb_log_files_in_group','2'
          'innodb_log_group_home_dir','./'
          'innodb_lru_scan_depth','1024'
          'innodb_max_bitmap_file_size','104857600'
          'innodb_max_changed_pages','1000000'
          'innodb_max_dirty_pages_pct','75.000000'
          'innodb_max_dirty_pages_pct_lwm','0.001000'
          'innodb_max_purge_lag','0'
          'innodb_max_purge_lag_delay','0'
          'innodb_mirrored_log_groups','1'
          'innodb_monitor_disable',''
          'innodb_monitor_enable',''
          'innodb_monitor_reset',''
          'innodb_monitor_reset_all',''
          'innodb_old_blocks_pct','37'
          'innodb_old_blocks_time','1000'
          'innodb_online_alter_log_max_size','134217728'
          'innodb_open_files','512'
          'innodb_optimize_fulltext_only','OFF'
          'innodb_page_size','16384'
          'innodb_print_all_deadlocks','OFF'
          'innodb_purge_batch_size','300'
          'innodb_purge_threads','1'
          'innodb_random_read_ahead','OFF'
          'innodb_read_ahead_threshold','56'
          'innodb_read_io_threads','64'
          'innodb_read_only','OFF'
          'innodb_replication_delay','0'
          'innodb_rollback_on_timeout','OFF'
          'innodb_rollback_segments','128'
          'innodb_sched_priority_cleaner','19'
          'innodb_show_locks_held','10'
          'innodb_show_verbose_locks','0'
          'innodb_simulate_comp_failures','0'
          'innodb_sort_buffer_size','1048576'
          'innodb_spin_wait_delay','6'
          'innodb_stats_auto_recalc','ON'
          'innodb_stats_method','nulls_equal'
          'innodb_stats_modified_counter','0'
          'innodb_stats_on_metadata','OFF'
          'innodb_stats_persistent','ON'
          'innodb_stats_persistent_sample_pages','20'
          'innodb_stats_sample_pages','8'
          'innodb_stats_traditional','ON'
          'innodb_stats_transient_sample_pages','8'
          'innodb_status_output','OFF'
          'innodb_status_output_locks','OFF'
          'innodb_strict_mode','OFF'
          'innodb_support_xa','ON'
          'innodb_sync_array_size','1'
          'innodb_sync_spin_loops','30'
          'innodb_table_locks','ON'
          'innodb_thread_concurrency','0'
          'innodb_thread_sleep_delay','10000'
          'innodb_track_changed_pages','OFF'
          'innodb_undo_directory','.'
          'innodb_undo_logs','128'
          'innodb_undo_tablespaces','0'
          'innodb_use_atomic_writes','OFF'
          'innodb_use_fallocate','OFF'
          'innodb_use_global_flush_log_at_trx_commit','ON'
          'innodb_use_native_aio','ON'
          'innodb_use_stacktrace','OFF'
          'innodb_use_sys_malloc','ON'
          'innodb_version','5.6.22-71.0'
          'innodb_write_io_threads','64'
          'insert_id','0'
          'interactive_timeout','28800'
          'join_buffer_size','131072'
          'join_buffer_space_limit','2097152'
          'join_cache_level','2'
          'keep_files_on_create','OFF'
          'key_buffer_size','134217728'
          'key_cache_age_threshold','300'
          'key_cache_block_size','1024'
          'key_cache_division_limit','100'
          'key_cache_file_hash_size','512'
          'key_cache_segments','0'
          'large_files_support','ON'
          'large_page_size','0'
          'large_pages','OFF'
          'last_gtid',''
          'last_insert_id','0'
          'lc_messages','en_US'
          'lc_messages_dir',''
          'lc_time_names','en_US'
          'license','GPL'
          'local_infile','ON'
          'lock_wait_timeout','31536000'
          'locked_in_memory','OFF'
          'log_bin','ON'
          'log_bin_trust_function_creators','OFF'
          'log_error','/var/log/mysqld.log'
          'log_output','FILE'
          'log_queries_not_using_indexes','OFF'
          'log_slave_updates','ON'
          'log_slow_filter','admin,filesort,filesort_on_disk,full_join,full_scan,query_cache,query_cache_miss,tmp_table,tmp_table_on_disk'
          'log_slow_rate_limit','1'
          'log_slow_verbosity',''
          'log_warnings','1'
          'long_query_time','10.000000'
          'low_priority_updates','OFF'
          'lower_case_file_system','OFF'
          'lower_case_table_names','1'
          'master_verify_checksum','OFF'
          'max_allowed_packet','33554432'
          'max_binlog_cache_size','18446744073709547520'
          'max_binlog_size','1073741824'
          'max_binlog_stmt_cache_size','18446744073709547520'
          'max_connect_errors','100'
          'max_connections','500'
          'max_delayed_threads','20'
          'max_error_count','64'
          'max_heap_table_size','67108864'
          'max_insert_delayed_threads','20'
          'max_join_size','18446744073709551615'
          'max_length_for_sort_data','1024'
          'max_long_data_size','33554432'
          'max_prepared_stmt_count','16382'
          'max_relay_log_size','1073741824'
          'max_seeks_for_key','4294967295'
          'max_sort_length','1024'
          'max_sp_recursion_depth','0'
          'max_tmp_tables','32'
          'max_user_connections','0'
          'max_write_lock_count','4294967295'
          'metadata_locks_cache_size','1024'
          'metadata_locks_hash_instances','8'
          'min_examined_row_limit','0'
          'mrr_buffer_size','262144'
          'multi_range_count','256'
          'myisam_block_size','1024'
          'myisam_data_pointer_size','6'
          'myisam_max_sort_file_size','9223372036853727232'
          'myisam_mmap_size','18446744073709551615'
          'myisam_recover_options','DEFAULT'
          'myisam_repair_threads','1'
          'myisam_sort_buffer_size','134216704'
          'myisam_stats_method','nulls_unequal'
          'myisam_use_mmap','OFF'
          'net_buffer_length','16384'
          'net_read_timeout','30'
          'net_retry_count','10'
          'net_write_timeout','60'
          'old','OFF'
          'old_alter_table','OFF'
          'old_mode',''
          'old_passwords','OFF'
          'open_files_limit','500005'
          'optimizer_prune_level','1'
          'optimizer_search_depth','62'
          'optimizer_selectivity_sampling_limit','100'
          'optimizer_switch','index_merge=on,index_merge_union=on,index_merge_sort_union=on,index_merge_intersection=on,index_merge_sort_intersection=off,engine_condition_pushdown=off,index_condition_pushdown=on,derived_merge=on,derived_with_keys=on,firstmatch=on,loosescan=on,materialization=on,in_to_exists=on,semijoin=on,partial_match_rowid_merge=on,partial_match_table_scan=on,subquery_cache=on,mrr=off,mrr_cost_based=off,mrr_sort_keys=off,outer_join_with_cache=on,semijoin_with_cache=on,join_cache_incremental=on,join_cache_hashed=on,join_cache_bka=on,optimize_join_buffer_size=off,table_elimination=on,extended_keys=on,exists_to_in=on'
          'optimizer_use_condition_selectivity','1'
          'performance_schema','ON'
          'performance_schema_accounts_size','100'
          'performance_schema_digests_size','10000'
          'performance_schema_events_stages_history_long_size','10000'
          'performance_schema_events_stages_history_size','10'
          'performance_schema_events_statements_history_long_size','10000'
          'performance_schema_events_statements_history_size','10'
          'performance_schema_events_waits_history_long_size','10000'
          'performance_schema_events_waits_history_size','10'
          'performance_schema_hosts_size','100'
          'performance_schema_max_cond_classes','80'
          'performance_schema_max_cond_instances','400900'
          'performance_schema_max_file_classes','50'
          'performance_schema_max_file_handles','32768'
          'performance_schema_max_file_instances','2154'
          'performance_schema_max_mutex_classes','200'
          'performance_schema_max_mutex_instances','605000'
          'performance_schema_max_rwlock_classes','40'
          'performance_schema_max_rwlock_instances','202800'
          'performance_schema_max_socket_classes','10'
          'performance_schema_max_socket_instances','200020'
          'performance_schema_max_stage_classes','150'
          'performance_schema_max_statement_classes','180'
          'performance_schema_max_table_handles','800'
          'performance_schema_max_table_instances','12500'
          'performance_schema_max_thread_classes','50'
          'performance_schema_max_thread_instances','200100'
          'performance_schema_session_connect_attrs_size','512'
          'performance_schema_setup_actors_size','100'
          'performance_schema_setup_objects_size','100'
          'performance_schema_users_size','100'
          'pid_file','/var/lib/mysql/optosql5.pid'
          'plugin_dir','/usr/lib64/mysql/plugin/'
          'plugin_maturity','unknown'
          'port','3306'
          'preload_buffer_size','32768'
          'profiling','OFF'
          'profiling_history_size','15'
          'progress_report_time','5'
          'protocol_version','10'
          'proxy_user',''
          'pseudo_slave_mode','OFF'
          'pseudo_thread_id','189143'
          'query_alloc_block_size','8192'
          'query_cache_limit','1048576'
          'query_cache_min_res_unit','4096'
          'query_cache_size','67108864'
          'query_cache_strip_comments','OFF'
          'query_cache_type','ON'
          'query_cache_wlock_invalidate','OFF'
          'query_prealloc_size','8192'
          'rand_seed1','0'
          'rand_seed2','0'
          'range_alloc_block_size','4096'
          'read_buffer_size','131072'
          'read_only','ON'
          'read_rnd_buffer_size','262144'
          'relay_log',''
          'relay_log_index',''
          'relay_log_info_file','relay-log.info'
          'relay_log_purge','ON'
          'relay_log_recovery','OFF'
          'relay_log_space_limit','0'
          'replicate_annotate_row_events','OFF'
          'replicate_do_db',''
          'replicate_do_table',''
          'replicate_events_marked_for_skip','replicate'
          'replicate_ignore_db',''
          'replicate_ignore_table',''
          'replicate_wild_do_table',''
          'replicate_wild_ignore_table',''
          'report_host',''
          'report_password',''
          'report_port','3306'
          'report_user',''
          'rowid_merge_buff_size','8388608'
          'rpl_recovery_rank','0'
          'secure_auth','OFF'
          'secure_file_priv',''
          'server_id','5'
          'skip_external_locking','ON'
          'skip_name_resolve','ON'
          'skip_networking','OFF'
          'skip_replication','OFF'
          'skip_show_database','OFF'
          'slave_compressed_protocol','OFF'
          'slave_ddl_exec_mode','IDEMPOTENT'
          'slave_domain_parallel_threads','4'
          'slave_exec_mode','STRICT'
          'slave_load_tmpdir','/tmp'
          'slave_max_allowed_packet','1073741824'
          'slave_net_timeout','3600'
          'slave_parallel_max_queued','131072'
          'slave_parallel_threads','12'
          'slave_skip_errors','OFF'
          'slave_sql_verify_checksum','ON'
          'slave_transaction_retries','10'
          'slave_type_conversions',''
          'slow_launch_time','2'
          'slow_query_log','ON'
          'slow_query_log_file','optosql5-slow.log'
          'socket','/var/lib/mysql/mysql.sock'
          'sort_buffer_size','2097152'
          'sql_auto_is_null','OFF'
          'sql_big_selects','ON'
          'sql_buffer_result','OFF'
          'sql_log_bin','ON'
          'sql_log_off','OFF'
          'sql_mode',''
          'sql_notes','ON'
          'sql_quote_show_create','ON'
          'sql_safe_updates','OFF'
          'sql_select_limit','18446744073709551615'
          'sql_slave_skip_counter','0'
          'sql_warnings','OFF'
          'ssl_ca',''
          'ssl_capath',''
          'ssl_cert',''
          'ssl_cipher',''
          'ssl_crl',''
          'ssl_crlpath',''
          'ssl_key',''
          'storage_engine','InnoDB'
          'stored_program_cache','256'
          'sync_binlog','0'
          'sync_frm','ON'
          'sync_master_info','0'
          'sync_relay_log','0'
          'sync_relay_log_info','0'
          'system_time_zone','CEST'
          'table_definition_cache','400'
          'table_open_cache','512'
          'thread_cache_size','4'
          'thread_concurrency','10'
          'thread_handling','one-thread-per-connection'
          'thread_pool_idle_timeout','60'
          'thread_pool_max_threads','500'
          'thread_pool_oversubscribe','3'
          'thread_pool_size','16'
          'thread_pool_stall_limit','500'
          'thread_stack','294912'
          'time_format','%H:%i:%s'
          'time_zone','SYSTEM'
          'timed_mutexes','OFF'
          'timestamp','1434011008.405971'
          'tmp_table_size','134217728'
          'tmpdir','/tmp'
          'transaction_alloc_block_size','8192'
          'transaction_prealloc_size','4096'
          'tx_isolation','REPEATABLE-READ'
          'tx_read_only','OFF'
          'unique_checks','ON'
          'updatable_views_with_limit','YES'
          'use_stat_tables','NEVER'
          'userstat','OFF'
          'version','10.0.16-MariaDB-log'
          'version_comment','MariaDB Server'
          'version_compile_machine','x86_64'
          'version_compile_os','Linux'
          'version_malloc_library','bundled jemalloc'
          'wait_timeout','28800'
          'warning_count','0'
          }}

          The error occurs about once a week during normal operation.

          If you need further information, or if you want us to test something, please let me know.

          Best regards,

          Thomas Mischke
          thomas.mischke Thomas Mischke made changes -
          Description Hello,

          we are having a random "duplicate key" problem with parallel replication. We were trying hard to reproduce it on our test servers as well, but currently it only happens on our live servers.

          The error we get is the following:

          {quote}150604 2:56:41 [ERROR] Slave SQL: Error 'Duplicate entry 'VerifyOrder_0-249' for key 'identry_idqueue'' on query. Default database: 'optovision'. Query: 'insert into `queueentry`(`identry`,`idqueue`,`entrydate`,`removedate`,`info`,`prio`,`reserved`,`ObjectContext`,`OriginHost`,`ReservingHost`,`UnreserveDate`,`ReservationCounter`,`OptimisticLockField`)values('VerifyOrder_0',249,'2015-06-04 02:56:40.687','2015-06-04 03:11:40.671','',0,null,'semaphore','172.16.185.22',null,null,0,0)', Gtid 0-3-454870355, Internal MariaDB error code: 1062
          150604 2:56:41 [Warning] Slave: Duplicate entry 'VerifyOrder_0-249' for key 'identry_idqueue' Error_code: 1062
          150604 2:56:41 [ERROR] Error running query, slave SQL thread aborted. Fix the problem, and restart the slave SQL thread with "SLAVE START". We stopped at log 'mysql-bin.007762' position 645226271; GTID position '77-3-552,0-3-454870353,3-3-2,2-3-469046,17-3-180,1-3-7'
          150604 2:56:41 [Note] Error reading relay log event: slave SQL thread was killed{quote}

          It is always the same table that is involved in this issue, and in all cases we analyzed in deep, it was the same value for the unique key to be written.

          The replication is based on one master and one slave, no restrictions on the data to be transferred, and no direct write requests to the slave. Once a day the slave is stopped for a short moment to synchronize the file system and take a backup on LVM level. At the moment we see no connection between this backup and the error (the point of time it happens seems to be independent)

          The table that is involved is defined as follows:

          {{
          CREATE TABLE `queueentry` (
          `OID` INT(11) NOT NULL AUTO_INCREMENT,
          `identry` VARCHAR(100) NOT NULL,
          `entrydate` DATETIME NULL DEFAULT NULL,
          `removedate` DATETIME NULL DEFAULT NULL,
          `info` VARCHAR(8096) NULL DEFAULT NULL,
          `prio` INT(11) NULL DEFAULT NULL,
          `reserved` DATETIME NULL DEFAULT NULL,
          `idqueue` INT(11) NOT NULL,
          `ObjectContext` VARCHAR(200) NULL DEFAULT NULL,
          `OriginHost` VARCHAR(200) NULL DEFAULT NULL,
          `ReservingHost` VARCHAR(200) NULL DEFAULT NULL,
          `UnreserveDate` DATETIME NULL DEFAULT NULL,
          `ReservationCounter` INT(10) UNSIGNED NOT NULL DEFAULT '0',
          `OptimisticLockField` INT(11) NULL DEFAULT NULL,
          PRIMARY KEY (`OID`),
          UNIQUE INDEX `identry_idqueue` (`identry`, `idqueue`),
          INDEX `idqueue` (`idqueue`),
          INDEX `entrydate_prio_reserved_idqueue` (`idqueue`, `reserved`, `prio`, `entrydate`),
          INDEX `removedate` (`removedate`),
          INDEX `entrydate` (`entrydate`),
          INDEX `UnreserveDate_ReservationCounter` (`ReservationCounter`, `UnreserveDate`)
          )
          COLLATE='utf8_general_ci'
          ENGINE=InnoDB
          AUTO_INCREMENT=8164332;
          }}

          We analyzed the binlog of the master in those cases and found the following:
          * Just before this insert statement there are two transactions bound together for parallel replication.
          * Both of these transactions work on the same table as the insert does.
          * The second of these transactions deletes the entry with the unique key, that will be written by the statement showing the duplicate key error.

          That is the following two statements are bound together:
          {{
          insert into `queueentry`(`identry`,`idqueue`,`entrydate`,`removedate`,`info`,`prio`,`reserved`,`ObjectContext`,`OriginHost`,`ReservingHost`,`UnreserveDate`,`ReservationCounter`,`OptimisticLockField`)values('SME_PrintLabel_0',249,'2015-06-04 02:56:40.671','2015-06-04 03:11:40.656','',0,null,'semaphore','172.16.185.21',null,null,0,0)
          delete from `queueentry` where ((`OID` = 6363790) and (`OptimisticLockField` = 0))
          }}

          And then this next transaction crashes:
          {{
          insert into `queueentry`(`identry`,`idqueue`,`entrydate`,`removedate`,`info`,`prio`,`reserved`,`ObjectContext`,`OriginHost`,`ReservingHost`,`UnreserveDate`,`ReservationCounter`,`OptimisticLockField`)values('VerifyOrder_0',249,'2015-06-04 02:56:40.687','2015-06-04 03:11:40.671','',0,null,'semaphore','172.16.185.22',null,null,0,0)
          }}

          Here is the (I hope) most important part of the binlog of the master:

          {{
          #150604 2:56:40 server id 3 end_log_pos 645225814 GTID 0-3-454870353 cid=5620557227
          /*!100001 SET @@session.gtid_seq_no=454870353*//*!*/;
          BEGIN
          /*!*/;
          # at 645225814
          #150604 2:56:40 server id 3 end_log_pos 645225842 Intvar
          SET INSERT_ID=6363805/*!*/;
          # at 645225842
          #150604 2:56:40 server id 3 end_log_pos 645226244 Query thread_id=19732488 exec_time=0 error_code=0
          SET TIMESTAMP=1433379400/*!*/;
          insert into `queueentry`(`identry`,`idqueue`,`entrydate`,`removedate`,`info`,`prio`,`reserved`,`ObjectContext`,`OriginHost`,`ReservingHost`,`UnreserveDate`,`ReservationCounter`,`OptimisticLockField`)values('SME_PrintLabel_0',249,'2015-06-04 02:56:40.671','2015-06-04 03:11:40.656','',0,null,'semaphore','172.16.185.21',null,null,0,0)
          /*!*/;
          # at 645226244
          #150604 2:56:40 server id 3 end_log_pos 645226271 Xid = 5620557224
          COMMIT/*!*/;
          # at 645226271
          #150604 2:56:40 server id 3 end_log_pos 645226311 GTID 0-3-454870354 cid=5620557227
          /*!100001 SET @@session.gtid_seq_no=454870354*//*!*/;
          BEGIN
          /*!*/;
          # at 645226311
          #150604 2:56:40 server id 3 end_log_pos 645226462 Query thread_id=19732489 exec_time=0 error_code=0
          SET TIMESTAMP=1433379400/*!*/;
          delete from `queueentry` where ((`OID` = 6363790) and (`OptimisticLockField` = 0))
          /*!*/;
          # at 645226462
          #150604 2:56:40 server id 3 end_log_pos 645226489 Xid = 5620557228
          COMMIT/*!*/;
          # at 645226489
          #150604 2:56:40 server id 3 end_log_pos 645226527 GTID 0-3-454870355
          /*!100001 SET @@session.gtid_seq_no=454870355*//*!*/;
          BEGIN
          /*!*/;
          # at 645226527
          #150604 2:56:40 server id 3 end_log_pos 645226555 Intvar
          SET INSERT_ID=6363806/*!*/;
          # at 645226555
          #150604 2:56:40 server id 3 end_log_pos 645226954 Query thread_id=19732488 exec_time=0 error_code=0
          SET TIMESTAMP=1433379400/*!*/;
          insert into `queueentry`(`identry`,`idqueue`,`entrydate`,`removedate`,`info`,`prio`,`reserved`,`ObjectContext`,`OriginHost`,`ReservingHost`,`UnreserveDate`,`ReservationCounter`,`OptimisticLockField`)values('VerifyOrder_0',249,'2015-06-04 02:56:40.687','2015-06-04 03:11:40.671','',0,null,'semaphore','172.16.185.22',null,null,0,0)
          /*!*/;
          # at 645226954
          #150604 2:56:40 server id 3 end_log_pos 645226981 Xid = 5620557238
          COMMIT/*!*/;
          # at 645226981
          }}

          The following additional observations we made:
          * The replication error only happens when the slave is at least one second behind the master. So it seems to us that there is somewhere a race condition at work which only happens when the slave is working very fast on the transactions from the master.
          * When this error happens, the command "stop slave" does not work any longer. We have to kill the database process, since some slave transactions are still waiting on each other.
          * After a restart of the database server we examine the table, and we cannot see the malicious key in the table any longer (even before the restart it is not there). We can restart the replication without further actions.
          * When we restart the replication, the slave is normally a few minutes behind the master. While catching up, we normally get the same error again.
          * As far as we can see, it is always the value 'VerifyOrder_0' that is causing the error, even though other values are written to that table as often as that value. We find this interesting, since it is - in alphabetical order - the largest value that is being stored in that table at the moment. I have only a rough understanding about indexes in databases, but I wonder if this could be related to an index reorganization due to the previous insert and delete statements.
          * The error vanishes if we set binlog_commit_wait_count=0 on the master (which is our current workaround).

          The variables on our slave are set as follows:

          {{
          'Variable_name','Value'
          'aria_block_size','8192'
          'aria_checkpoint_interval','30'
          'aria_checkpoint_log_activity','1048576'
          'aria_force_start_after_recovery_failures','0'
          'aria_group_commit','none'
          'aria_group_commit_interval','0'
          'aria_log_file_size','1073741824'
          'aria_log_purge_type','immediate'
          'aria_max_sort_file_size','9223372036853727232'
          'aria_page_checksum','ON'
          'aria_pagecache_age_threshold','300'
          'aria_pagecache_buffer_size','134217728'
          'aria_pagecache_division_limit','100'
          'aria_pagecache_file_hash_size','512'
          'aria_recover','NORMAL'
          'aria_repair_threads','1'
          'aria_sort_buffer_size','268434432'
          'aria_stats_method','nulls_unequal'
          'aria_sync_log_dir','NEWFILE'
          'aria_used_for_temp_tables','ON'
          'auto_increment_increment','1'
          'auto_increment_offset','1'
          'autocommit','ON'
          'automatic_sp_privileges','ON'
          'back_log','150'
          'basedir','/usr'
          'big_tables','OFF'
          'binlog_annotate_row_events','OFF'
          'binlog_cache_size','32768'
          'binlog_checksum','NONE'
          'binlog_commit_wait_count','0'
          'binlog_commit_wait_usec','100000'
          'binlog_direct_non_transactional_updates','OFF'
          'binlog_format','MIXED'
          'binlog_optimize_thread_scheduling','ON'
          'binlog_stmt_cache_size','32768'
          'bulk_insert_buffer_size','8388608'
          'character_set_client','utf8'
          'character_set_connection','utf8'
          'character_set_database','utf8'
          'character_set_filesystem','binary'
          'character_set_results','utf8'
          'character_set_server','utf8'
          'character_set_system','utf8'
          'character_sets_dir','/usr/share/mysql/charsets/'
          'collation_connection','utf8_general_ci'
          'collation_database','utf8_general_ci'
          'collation_server','utf8_unicode_ci'
          'completion_type','NO_CHAIN'
          'concurrent_insert','AUTO'
          'connect_timeout','10'
          'datadir','/var/lib/mysql/'
          'date_format','%Y-%m-%d'
          'datetime_format','%Y-%m-%d %H:%i:%s'
          'deadlock_search_depth_long','15'
          'deadlock_search_depth_short','4'
          'deadlock_timeout_long','50000000'
          'deadlock_timeout_short','10000'
          'debug_no_thread_alarm','OFF'
          'default_master_connection',''
          'default_regex_flags',''
          'default_storage_engine','InnoDB'
          'default_week_format','0'
          'delay_key_write','ON'
          'delayed_insert_limit','100'
          'delayed_insert_timeout','300'
          'delayed_queue_size','1000'
          'div_precision_increment','4'
          'error_count','0'
          'event_scheduler','OFF'
          'expensive_subquery_limit','100'
          'expire_logs_days','5'
          'external_user',''
          'extra_max_connections','1'
          'extra_port','0'
          'flush','OFF'
          'flush_time','0'
          'foreign_key_checks','ON'
          'ft_boolean_syntax','+ -><()~*:""&|'
          'ft_max_word_len','84'
          'ft_min_word_len','4'
          'ft_query_expansion_limit','20'
          'ft_stopword_file','(built-in)'
          'general_log','OFF'
          'general_log_file','optosql5.log'
          'group_concat_max_len','1024'
          'gtid_binlog_pos','0-3-481964408,3-3-2,2-3-504931'
          'gtid_binlog_state','0-5-479038918,0-3-481964408,3-3-2,2-3-504931'
          'gtid_current_pos','77-3-552,0-3-481964408,3-3-2,2-3-504931,17-3-180,1-3-7'
          'gtid_domain_id','0'
          'gtid_ignore_duplicates','OFF'
          'gtid_seq_no','0'
          'gtid_slave_pos','77-3-552,0-3-481964408,3-3-2,2-3-504931,17-3-180,1-3-7'
          'gtid_strict_mode','OFF'
          'have_compress','YES'
          'have_crypt','YES'
          'have_dynamic_loading','YES'
          'have_geometry','YES'
          'have_openssl','YES'
          'have_profiling','YES'
          'have_query_cache','YES'
          'have_rtree_keys','YES'
          'have_ssl','DISABLED'
          'have_symlink','DISABLED'
          'histogram_size','0'
          'histogram_type','SINGLE_PREC_HB'
          'host_cache_size','128'
          'hostname','optosql5'
          'identity','0'
          'ignore_builtin_innodb','OFF'
          'ignore_db_dirs',''
          'in_transaction','0'
          'init_connect',''
          'init_file',''
          'init_slave',''
          'innodb_adaptive_flushing','ON'
          'innodb_adaptive_flushing_lwm','10.000000'
          'innodb_adaptive_hash_index','ON'
          'innodb_adaptive_hash_index_partitions','1'
          'innodb_adaptive_max_sleep_delay','150000'
          'innodb_additional_mem_pool_size','8388608'
          'innodb_api_bk_commit_interval','5'
          'innodb_api_disable_rowlock','OFF'
          'innodb_api_enable_binlog','OFF'
          'innodb_api_enable_mdl','OFF'
          'innodb_api_trx_level','0'
          'innodb_autoextend_increment','64'
          'innodb_autoinc_lock_mode','1'
          'innodb_buffer_pool_dump_at_shutdown','OFF'
          'innodb_buffer_pool_dump_now','OFF'
          'innodb_buffer_pool_filename','ib_buffer_pool'
          'innodb_buffer_pool_instances','8'
          'innodb_buffer_pool_load_abort','OFF'
          'innodb_buffer_pool_load_at_startup','OFF'
          'innodb_buffer_pool_load_now','OFF'
          'innodb_buffer_pool_populate','OFF'
          'innodb_buffer_pool_size','34359738368'
          'innodb_change_buffer_max_size','25'
          'innodb_change_buffering','all'
          'innodb_checksum_algorithm','innodb'
          'innodb_checksums','ON'
          'innodb_cleaner_lsn_age_factor','high_checkpoint'
          'innodb_cmp_per_index_enabled','OFF'
          'innodb_commit_concurrency','0'
          'innodb_compression_failure_threshold_pct','5'
          'innodb_compression_level','6'
          'innodb_compression_pad_pct_max','50'
          'innodb_concurrency_tickets','5000'
          'innodb_corrupt_table_action','assert'
          'innodb_data_file_path','ibdata1:12M:autoextend'
          'innodb_data_home_dir',''
          'innodb_disable_sort_file_cache','OFF'
          'innodb_doublewrite','ON'
          'innodb_empty_free_list_algorithm','backoff'
          'innodb_fake_changes','OFF'
          'innodb_fast_shutdown','1'
          'innodb_file_format','Antelope'
          'innodb_file_format_check','ON'
          'innodb_file_format_max','Antelope'
          'innodb_file_per_table','ON'
          'innodb_flush_log_at_timeout','1'
          'innodb_flush_log_at_trx_commit','1'
          'innodb_flush_method',''
          'innodb_flush_neighbors','1'
          'innodb_flushing_avg_loops','30'
          'innodb_force_load_corrupted','OFF'
          'innodb_force_recovery','0'
          'innodb_foreground_preflush','exponential_backoff'
          'innodb_ft_aux_table',''
          'innodb_ft_cache_size','8000000'
          'innodb_ft_enable_diag_print','OFF'
          'innodb_ft_enable_stopword','ON'
          'innodb_ft_max_token_size','84'
          'innodb_ft_min_token_size','3'
          'innodb_ft_num_word_optimize','2000'
          'innodb_ft_result_cache_limit','2000000000'
          'innodb_ft_server_stopword_table',''
          'innodb_ft_sort_pll_degree','2'
          'innodb_ft_total_cache_size','640000000'
          'innodb_ft_user_stopword_table',''
          'innodb_io_capacity','200'
          'innodb_io_capacity_max','2000'
          'innodb_kill_idle_transaction','0'
          'innodb_large_prefix','OFF'
          'innodb_lock_wait_timeout','50'
          'innodb_locking_fake_changes','ON'
          'innodb_locks_unsafe_for_binlog','OFF'
          'innodb_log_arch_dir','./'
          'innodb_log_arch_expire_sec','0'
          'innodb_log_archive','OFF'
          'innodb_log_block_size','512'
          'innodb_log_buffer_size','8388608'
          'innodb_log_checksum_algorithm','innodb'
          'innodb_log_compressed_pages','ON'
          'innodb_log_file_size','104857600'
          'innodb_log_files_in_group','2'
          'innodb_log_group_home_dir','./'
          'innodb_lru_scan_depth','1024'
          'innodb_max_bitmap_file_size','104857600'
          'innodb_max_changed_pages','1000000'
          'innodb_max_dirty_pages_pct','75.000000'
          'innodb_max_dirty_pages_pct_lwm','0.001000'
          'innodb_max_purge_lag','0'
          'innodb_max_purge_lag_delay','0'
          'innodb_mirrored_log_groups','1'
          'innodb_monitor_disable',''
          'innodb_monitor_enable',''
          'innodb_monitor_reset',''
          'innodb_monitor_reset_all',''
          'innodb_old_blocks_pct','37'
          'innodb_old_blocks_time','1000'
          'innodb_online_alter_log_max_size','134217728'
          'innodb_open_files','512'
          'innodb_optimize_fulltext_only','OFF'
          'innodb_page_size','16384'
          'innodb_print_all_deadlocks','OFF'
          'innodb_purge_batch_size','300'
          'innodb_purge_threads','1'
          'innodb_random_read_ahead','OFF'
          'innodb_read_ahead_threshold','56'
          'innodb_read_io_threads','64'
          'innodb_read_only','OFF'
          'innodb_replication_delay','0'
          'innodb_rollback_on_timeout','OFF'
          'innodb_rollback_segments','128'
          'innodb_sched_priority_cleaner','19'
          'innodb_show_locks_held','10'
          'innodb_show_verbose_locks','0'
          'innodb_simulate_comp_failures','0'
          'innodb_sort_buffer_size','1048576'
          'innodb_spin_wait_delay','6'
          'innodb_stats_auto_recalc','ON'
          'innodb_stats_method','nulls_equal'
          'innodb_stats_modified_counter','0'
          'innodb_stats_on_metadata','OFF'
          'innodb_stats_persistent','ON'
          'innodb_stats_persistent_sample_pages','20'
          'innodb_stats_sample_pages','8'
          'innodb_stats_traditional','ON'
          'innodb_stats_transient_sample_pages','8'
          'innodb_status_output','OFF'
          'innodb_status_output_locks','OFF'
          'innodb_strict_mode','OFF'
          'innodb_support_xa','ON'
          'innodb_sync_array_size','1'
          'innodb_sync_spin_loops','30'
          'innodb_table_locks','ON'
          'innodb_thread_concurrency','0'
          'innodb_thread_sleep_delay','10000'
          'innodb_track_changed_pages','OFF'
          'innodb_undo_directory','.'
          'innodb_undo_logs','128'
          'innodb_undo_tablespaces','0'
          'innodb_use_atomic_writes','OFF'
          'innodb_use_fallocate','OFF'
          'innodb_use_global_flush_log_at_trx_commit','ON'
          'innodb_use_native_aio','ON'
          'innodb_use_stacktrace','OFF'
          'innodb_use_sys_malloc','ON'
          'innodb_version','5.6.22-71.0'
          'innodb_write_io_threads','64'
          'insert_id','0'
          'interactive_timeout','28800'
          'join_buffer_size','131072'
          'join_buffer_space_limit','2097152'
          'join_cache_level','2'
          'keep_files_on_create','OFF'
          'key_buffer_size','134217728'
          'key_cache_age_threshold','300'
          'key_cache_block_size','1024'
          'key_cache_division_limit','100'
          'key_cache_file_hash_size','512'
          'key_cache_segments','0'
          'large_files_support','ON'
          'large_page_size','0'
          'large_pages','OFF'
          'last_gtid',''
          'last_insert_id','0'
          'lc_messages','en_US'
          'lc_messages_dir',''
          'lc_time_names','en_US'
          'license','GPL'
          'local_infile','ON'
          'lock_wait_timeout','31536000'
          'locked_in_memory','OFF'
          'log_bin','ON'
          'log_bin_trust_function_creators','OFF'
          'log_error','/var/log/mysqld.log'
          'log_output','FILE'
          'log_queries_not_using_indexes','OFF'
          'log_slave_updates','ON'
          'log_slow_filter','admin,filesort,filesort_on_disk,full_join,full_scan,query_cache,query_cache_miss,tmp_table,tmp_table_on_disk'
          'log_slow_rate_limit','1'
          'log_slow_verbosity',''
          'log_warnings','1'
          'long_query_time','10.000000'
          'low_priority_updates','OFF'
          'lower_case_file_system','OFF'
          'lower_case_table_names','1'
          'master_verify_checksum','OFF'
          'max_allowed_packet','33554432'
          'max_binlog_cache_size','18446744073709547520'
          'max_binlog_size','1073741824'
          'max_binlog_stmt_cache_size','18446744073709547520'
          'max_connect_errors','100'
          'max_connections','500'
          'max_delayed_threads','20'
          'max_error_count','64'
          'max_heap_table_size','67108864'
          'max_insert_delayed_threads','20'
          'max_join_size','18446744073709551615'
          'max_length_for_sort_data','1024'
          'max_long_data_size','33554432'
          'max_prepared_stmt_count','16382'
          'max_relay_log_size','1073741824'
          'max_seeks_for_key','4294967295'
          'max_sort_length','1024'
          'max_sp_recursion_depth','0'
          'max_tmp_tables','32'
          'max_user_connections','0'
          'max_write_lock_count','4294967295'
          'metadata_locks_cache_size','1024'
          'metadata_locks_hash_instances','8'
          'min_examined_row_limit','0'
          'mrr_buffer_size','262144'
          'multi_range_count','256'
          'myisam_block_size','1024'
          'myisam_data_pointer_size','6'
          'myisam_max_sort_file_size','9223372036853727232'
          'myisam_mmap_size','18446744073709551615'
          'myisam_recover_options','DEFAULT'
          'myisam_repair_threads','1'
          'myisam_sort_buffer_size','134216704'
          'myisam_stats_method','nulls_unequal'
          'myisam_use_mmap','OFF'
          'net_buffer_length','16384'
          'net_read_timeout','30'
          'net_retry_count','10'
          'net_write_timeout','60'
          'old','OFF'
          'old_alter_table','OFF'
          'old_mode',''
          'old_passwords','OFF'
          'open_files_limit','500005'
          'optimizer_prune_level','1'
          'optimizer_search_depth','62'
          'optimizer_selectivity_sampling_limit','100'
          'optimizer_switch','index_merge=on,index_merge_union=on,index_merge_sort_union=on,index_merge_intersection=on,index_merge_sort_intersection=off,engine_condition_pushdown=off,index_condition_pushdown=on,derived_merge=on,derived_with_keys=on,firstmatch=on,loosescan=on,materialization=on,in_to_exists=on,semijoin=on,partial_match_rowid_merge=on,partial_match_table_scan=on,subquery_cache=on,mrr=off,mrr_cost_based=off,mrr_sort_keys=off,outer_join_with_cache=on,semijoin_with_cache=on,join_cache_incremental=on,join_cache_hashed=on,join_cache_bka=on,optimize_join_buffer_size=off,table_elimination=on,extended_keys=on,exists_to_in=on'
          'optimizer_use_condition_selectivity','1'
          'performance_schema','ON'
          'performance_schema_accounts_size','100'
          'performance_schema_digests_size','10000'
          'performance_schema_events_stages_history_long_size','10000'
          'performance_schema_events_stages_history_size','10'
          'performance_schema_events_statements_history_long_size','10000'
          'performance_schema_events_statements_history_size','10'
          'performance_schema_events_waits_history_long_size','10000'
          'performance_schema_events_waits_history_size','10'
          'performance_schema_hosts_size','100'
          'performance_schema_max_cond_classes','80'
          'performance_schema_max_cond_instances','400900'
          'performance_schema_max_file_classes','50'
          'performance_schema_max_file_handles','32768'
          'performance_schema_max_file_instances','2154'
          'performance_schema_max_mutex_classes','200'
          'performance_schema_max_mutex_instances','605000'
          'performance_schema_max_rwlock_classes','40'
          'performance_schema_max_rwlock_instances','202800'
          'performance_schema_max_socket_classes','10'
          'performance_schema_max_socket_instances','200020'
          'performance_schema_max_stage_classes','150'
          'performance_schema_max_statement_classes','180'
          'performance_schema_max_table_handles','800'
          'performance_schema_max_table_instances','12500'
          'performance_schema_max_thread_classes','50'
          'performance_schema_max_thread_instances','200100'
          'performance_schema_session_connect_attrs_size','512'
          'performance_schema_setup_actors_size','100'
          'performance_schema_setup_objects_size','100'
          'performance_schema_users_size','100'
          'pid_file','/var/lib/mysql/optosql5.pid'
          'plugin_dir','/usr/lib64/mysql/plugin/'
          'plugin_maturity','unknown'
          'port','3306'
          'preload_buffer_size','32768'
          'profiling','OFF'
          'profiling_history_size','15'
          'progress_report_time','5'
          'protocol_version','10'
          'proxy_user',''
          'pseudo_slave_mode','OFF'
          'pseudo_thread_id','189143'
          'query_alloc_block_size','8192'
          'query_cache_limit','1048576'
          'query_cache_min_res_unit','4096'
          'query_cache_size','67108864'
          'query_cache_strip_comments','OFF'
          'query_cache_type','ON'
          'query_cache_wlock_invalidate','OFF'
          'query_prealloc_size','8192'
          'rand_seed1','0'
          'rand_seed2','0'
          'range_alloc_block_size','4096'
          'read_buffer_size','131072'
          'read_only','ON'
          'read_rnd_buffer_size','262144'
          'relay_log',''
          'relay_log_index',''
          'relay_log_info_file','relay-log.info'
          'relay_log_purge','ON'
          'relay_log_recovery','OFF'
          'relay_log_space_limit','0'
          'replicate_annotate_row_events','OFF'
          'replicate_do_db',''
          'replicate_do_table',''
          'replicate_events_marked_for_skip','replicate'
          'replicate_ignore_db',''
          'replicate_ignore_table',''
          'replicate_wild_do_table',''
          'replicate_wild_ignore_table',''
          'report_host',''
          'report_password',''
          'report_port','3306'
          'report_user',''
          'rowid_merge_buff_size','8388608'
          'rpl_recovery_rank','0'
          'secure_auth','OFF'
          'secure_file_priv',''
          'server_id','5'
          'skip_external_locking','ON'
          'skip_name_resolve','ON'
          'skip_networking','OFF'
          'skip_replication','OFF'
          'skip_show_database','OFF'
          'slave_compressed_protocol','OFF'
          'slave_ddl_exec_mode','IDEMPOTENT'
          'slave_domain_parallel_threads','4'
          'slave_exec_mode','STRICT'
          'slave_load_tmpdir','/tmp'
          'slave_max_allowed_packet','1073741824'
          'slave_net_timeout','3600'
          'slave_parallel_max_queued','131072'
          'slave_parallel_threads','12'
          'slave_skip_errors','OFF'
          'slave_sql_verify_checksum','ON'
          'slave_transaction_retries','10'
          'slave_type_conversions',''
          'slow_launch_time','2'
          'slow_query_log','ON'
          'slow_query_log_file','optosql5-slow.log'
          'socket','/var/lib/mysql/mysql.sock'
          'sort_buffer_size','2097152'
          'sql_auto_is_null','OFF'
          'sql_big_selects','ON'
          'sql_buffer_result','OFF'
          'sql_log_bin','ON'
          'sql_log_off','OFF'
          'sql_mode',''
          'sql_notes','ON'
          'sql_quote_show_create','ON'
          'sql_safe_updates','OFF'
          'sql_select_limit','18446744073709551615'
          'sql_slave_skip_counter','0'
          'sql_warnings','OFF'
          'ssl_ca',''
          'ssl_capath',''
          'ssl_cert',''
          'ssl_cipher',''
          'ssl_crl',''
          'ssl_crlpath',''
          'ssl_key',''
          'storage_engine','InnoDB'
          'stored_program_cache','256'
          'sync_binlog','0'
          'sync_frm','ON'
          'sync_master_info','0'
          'sync_relay_log','0'
          'sync_relay_log_info','0'
          'system_time_zone','CEST'
          'table_definition_cache','400'
          'table_open_cache','512'
          'thread_cache_size','4'
          'thread_concurrency','10'
          'thread_handling','one-thread-per-connection'
          'thread_pool_idle_timeout','60'
          'thread_pool_max_threads','500'
          'thread_pool_oversubscribe','3'
          'thread_pool_size','16'
          'thread_pool_stall_limit','500'
          'thread_stack','294912'
          'time_format','%H:%i:%s'
          'time_zone','SYSTEM'
          'timed_mutexes','OFF'
          'timestamp','1434011008.405971'
          'tmp_table_size','134217728'
          'tmpdir','/tmp'
          'transaction_alloc_block_size','8192'
          'transaction_prealloc_size','4096'
          'tx_isolation','REPEATABLE-READ'
          'tx_read_only','OFF'
          'unique_checks','ON'
          'updatable_views_with_limit','YES'
          'use_stat_tables','NEVER'
          'userstat','OFF'
          'version','10.0.16-MariaDB-log'
          'version_comment','MariaDB Server'
          'version_compile_machine','x86_64'
          'version_compile_os','Linux'
          'version_malloc_library','bundled jemalloc'
          'wait_timeout','28800'
          'warning_count','0'
          }}

          The error occurs about once a week during normal operation.

          If you need further information, or if you want us to test something, please let me know.

          Best regards,

          Thomas Mischke
          Hello,

          we are having a random "duplicate key" problem with parallel replication. We were trying hard to reproduce it on our test servers as well, but currently it only happens on our live servers.

          The error we get is the following:

          bq. 150604 2:56:41 [ERROR] Slave SQL: Error 'Duplicate entry 'VerifyOrder_0-249' for key 'identry_idqueue'' on query. Default database: 'optovision'. Query: 'insert into `queueentry`(`identry`,`idqueue`,`entrydate`,`removedate`,`info`,`prio`,`reserved`,`ObjectContext`,`OriginHost`,`ReservingHost`,`UnreserveDate`,`ReservationCounter`,`OptimisticLockField`)values('VerifyOrder_0',249,'2015-06-04 02:56:40.687','2015-06-04 03:11:40.671','',0,null,'semaphore','172.16.185.22',null,null,0,0)', Gtid 0-3-454870355, Internal MariaDB error code: 1062
          bq. 150604 2:56:41 [Warning] Slave: Duplicate entry 'VerifyOrder_0-249' for key 'identry_idqueue' Error_code: 1062
          bq. 150604 2:56:41 [ERROR] Error running query, slave SQL thread aborted. Fix the problem, and restart the slave SQL thread with "SLAVE START". We stopped at log 'mysql-bin.007762' position 645226271; GTID position '77-3-552,0-3-454870353,3-3-2,2-3-469046,17-3-180,1-3-7'
          bq. 150604 2:56:41 [Note] Error reading relay log event: slave SQL thread was killed

          It is always the same table that is involved in this issue, and in all cases we analyzed in deep, it was the same value for the unique key to be written.

          The replication is based on one master and one slave, no restrictions on the data to be transferred, and no direct write requests to the slave. Once a day the slave is stopped for a short moment to synchronize the file system and take a backup on LVM level. At the moment we see no connection between this backup and the error (the point of time it happens seems to be independent)

          The table that is involved is defined as follows:

          {{
          CREATE TABLE `queueentry` (
          `OID` INT(11) NOT NULL AUTO_INCREMENT,
          `identry` VARCHAR(100) NOT NULL,
          `entrydate` DATETIME NULL DEFAULT NULL,
          `removedate` DATETIME NULL DEFAULT NULL,
          `info` VARCHAR(8096) NULL DEFAULT NULL,
          `prio` INT(11) NULL DEFAULT NULL,
          `reserved` DATETIME NULL DEFAULT NULL,
          `idqueue` INT(11) NOT NULL,
          `ObjectContext` VARCHAR(200) NULL DEFAULT NULL,
          `OriginHost` VARCHAR(200) NULL DEFAULT NULL,
          `ReservingHost` VARCHAR(200) NULL DEFAULT NULL,
          `UnreserveDate` DATETIME NULL DEFAULT NULL,
          `ReservationCounter` INT(10) UNSIGNED NOT NULL DEFAULT '0',
          `OptimisticLockField` INT(11) NULL DEFAULT NULL,
          PRIMARY KEY (`OID`),
          UNIQUE INDEX `identry_idqueue` (`identry`, `idqueue`),
          INDEX `idqueue` (`idqueue`),
          INDEX `entrydate_prio_reserved_idqueue` (`idqueue`, `reserved`, `prio`, `entrydate`),
          INDEX `removedate` (`removedate`),
          INDEX `entrydate` (`entrydate`),
          INDEX `UnreserveDate_ReservationCounter` (`ReservationCounter`, `UnreserveDate`)
          )
          COLLATE='utf8_general_ci'
          ENGINE=InnoDB
          AUTO_INCREMENT=8164332;
          }}

          We analyzed the binlog of the master in those cases and found the following:
          * Just before this insert statement there are two transactions bound together for parallel replication.
          * Both of these transactions work on the same table as the insert does.
          * The second of these transactions deletes the entry with the unique key, that will be written by the statement showing the duplicate key error.

          That is the following two statements are bound together:
          {{
          insert into `queueentry`(`identry`,`idqueue`,`entrydate`,`removedate`,`info`,`prio`,`reserved`,`ObjectContext`,`OriginHost`,`ReservingHost`,`UnreserveDate`,`ReservationCounter`,`OptimisticLockField`)values('SME_PrintLabel_0',249,'2015-06-04 02:56:40.671','2015-06-04 03:11:40.656','',0,null,'semaphore','172.16.185.21',null,null,0,0)
          delete from `queueentry` where ((`OID` = 6363790) and (`OptimisticLockField` = 0))
          }}

          And then this next transaction crashes:
          {{
          insert into `queueentry`(`identry`,`idqueue`,`entrydate`,`removedate`,`info`,`prio`,`reserved`,`ObjectContext`,`OriginHost`,`ReservingHost`,`UnreserveDate`,`ReservationCounter`,`OptimisticLockField`)values('VerifyOrder_0',249,'2015-06-04 02:56:40.687','2015-06-04 03:11:40.671','',0,null,'semaphore','172.16.185.22',null,null,0,0)
          }}

          Here is the (I hope) most important part of the binlog of the master:

          {{
          #150604 2:56:40 server id 3 end_log_pos 645225814 GTID 0-3-454870353 cid=5620557227
          /*!100001 SET @@session.gtid_seq_no=454870353*//*!*/;
          BEGIN
          /*!*/;
          # at 645225814
          #150604 2:56:40 server id 3 end_log_pos 645225842 Intvar
          SET INSERT_ID=6363805/*!*/;
          # at 645225842
          #150604 2:56:40 server id 3 end_log_pos 645226244 Query thread_id=19732488 exec_time=0 error_code=0
          SET TIMESTAMP=1433379400/*!*/;
          insert into `queueentry`(`identry`,`idqueue`,`entrydate`,`removedate`,`info`,`prio`,`reserved`,`ObjectContext`,`OriginHost`,`ReservingHost`,`UnreserveDate`,`ReservationCounter`,`OptimisticLockField`)values('SME_PrintLabel_0',249,'2015-06-04 02:56:40.671','2015-06-04 03:11:40.656','',0,null,'semaphore','172.16.185.21',null,null,0,0)
          /*!*/;
          # at 645226244
          #150604 2:56:40 server id 3 end_log_pos 645226271 Xid = 5620557224
          COMMIT/*!*/;
          # at 645226271
          #150604 2:56:40 server id 3 end_log_pos 645226311 GTID 0-3-454870354 cid=5620557227
          /*!100001 SET @@session.gtid_seq_no=454870354*//*!*/;
          BEGIN
          /*!*/;
          # at 645226311
          #150604 2:56:40 server id 3 end_log_pos 645226462 Query thread_id=19732489 exec_time=0 error_code=0
          SET TIMESTAMP=1433379400/*!*/;
          delete from `queueentry` where ((`OID` = 6363790) and (`OptimisticLockField` = 0))
          /*!*/;
          # at 645226462
          #150604 2:56:40 server id 3 end_log_pos 645226489 Xid = 5620557228
          COMMIT/*!*/;
          # at 645226489
          #150604 2:56:40 server id 3 end_log_pos 645226527 GTID 0-3-454870355
          /*!100001 SET @@session.gtid_seq_no=454870355*//*!*/;
          BEGIN
          /*!*/;
          # at 645226527
          #150604 2:56:40 server id 3 end_log_pos 645226555 Intvar
          SET INSERT_ID=6363806/*!*/;
          # at 645226555
          #150604 2:56:40 server id 3 end_log_pos 645226954 Query thread_id=19732488 exec_time=0 error_code=0
          SET TIMESTAMP=1433379400/*!*/;
          insert into `queueentry`(`identry`,`idqueue`,`entrydate`,`removedate`,`info`,`prio`,`reserved`,`ObjectContext`,`OriginHost`,`ReservingHost`,`UnreserveDate`,`ReservationCounter`,`OptimisticLockField`)values('VerifyOrder_0',249,'2015-06-04 02:56:40.687','2015-06-04 03:11:40.671','',0,null,'semaphore','172.16.185.22',null,null,0,0)
          /*!*/;
          # at 645226954
          #150604 2:56:40 server id 3 end_log_pos 645226981 Xid = 5620557238
          COMMIT/*!*/;
          # at 645226981
          }}

          The following additional observations we made:
          * The replication error only happens when the slave is at least one second behind the master. So it seems to us that there is somewhere a race condition at work which only happens when the slave is working very fast on the transactions from the master.
          * When this error happens, the command "stop slave" does not work any longer. We have to kill the database process, since some slave transactions are still waiting on each other.
          * After a restart of the database server we examine the table, and we cannot see the malicious key in the table any longer (even before the restart it is not there). We can restart the replication without further actions.
          * When we restart the replication, the slave is normally a few minutes behind the master. While catching up, we normally get the same error again.
          * As far as we can see, it is always the value 'VerifyOrder_0' that is causing the error, even though other values are written to that table as often as that value. We find this interesting, since it is - in alphabetical order - the largest value that is being stored in that table at the moment. I have only a rough understanding about indexes in databases, but I wonder if this could be related to an index reorganization due to the previous insert and delete statements.
          * The error vanishes if we set binlog_commit_wait_count=0 on the master (which is our current workaround).

          The variables on our slave are set as follows:

          {{
          'Variable_name','Value'
          'aria_block_size','8192'
          'aria_checkpoint_interval','30'
          'aria_checkpoint_log_activity','1048576'
          'aria_force_start_after_recovery_failures','0'
          'aria_group_commit','none'
          'aria_group_commit_interval','0'
          'aria_log_file_size','1073741824'
          'aria_log_purge_type','immediate'
          'aria_max_sort_file_size','9223372036853727232'
          'aria_page_checksum','ON'
          'aria_pagecache_age_threshold','300'
          'aria_pagecache_buffer_size','134217728'
          'aria_pagecache_division_limit','100'
          'aria_pagecache_file_hash_size','512'
          'aria_recover','NORMAL'
          'aria_repair_threads','1'
          'aria_sort_buffer_size','268434432'
          'aria_stats_method','nulls_unequal'
          'aria_sync_log_dir','NEWFILE'
          'aria_used_for_temp_tables','ON'
          'auto_increment_increment','1'
          'auto_increment_offset','1'
          'autocommit','ON'
          'automatic_sp_privileges','ON'
          'back_log','150'
          'basedir','/usr'
          'big_tables','OFF'
          'binlog_annotate_row_events','OFF'
          'binlog_cache_size','32768'
          'binlog_checksum','NONE'
          'binlog_commit_wait_count','0'
          'binlog_commit_wait_usec','100000'
          'binlog_direct_non_transactional_updates','OFF'
          'binlog_format','MIXED'
          'binlog_optimize_thread_scheduling','ON'
          'binlog_stmt_cache_size','32768'
          'bulk_insert_buffer_size','8388608'
          'character_set_client','utf8'
          'character_set_connection','utf8'
          'character_set_database','utf8'
          'character_set_filesystem','binary'
          'character_set_results','utf8'
          'character_set_server','utf8'
          'character_set_system','utf8'
          'character_sets_dir','/usr/share/mysql/charsets/'
          'collation_connection','utf8_general_ci'
          'collation_database','utf8_general_ci'
          'collation_server','utf8_unicode_ci'
          'completion_type','NO_CHAIN'
          'concurrent_insert','AUTO'
          'connect_timeout','10'
          'datadir','/var/lib/mysql/'
          'date_format','%Y-%m-%d'
          'datetime_format','%Y-%m-%d %H:%i:%s'
          'deadlock_search_depth_long','15'
          'deadlock_search_depth_short','4'
          'deadlock_timeout_long','50000000'
          'deadlock_timeout_short','10000'
          'debug_no_thread_alarm','OFF'
          'default_master_connection',''
          'default_regex_flags',''
          'default_storage_engine','InnoDB'
          'default_week_format','0'
          'delay_key_write','ON'
          'delayed_insert_limit','100'
          'delayed_insert_timeout','300'
          'delayed_queue_size','1000'
          'div_precision_increment','4'
          'error_count','0'
          'event_scheduler','OFF'
          'expensive_subquery_limit','100'
          'expire_logs_days','5'
          'external_user',''
          'extra_max_connections','1'
          'extra_port','0'
          'flush','OFF'
          'flush_time','0'
          'foreign_key_checks','ON'
          'ft_boolean_syntax','+ -><()~*:""&|'
          'ft_max_word_len','84'
          'ft_min_word_len','4'
          'ft_query_expansion_limit','20'
          'ft_stopword_file','(built-in)'
          'general_log','OFF'
          'general_log_file','optosql5.log'
          'group_concat_max_len','1024'
          'gtid_binlog_pos','0-3-481964408,3-3-2,2-3-504931'
          'gtid_binlog_state','0-5-479038918,0-3-481964408,3-3-2,2-3-504931'
          'gtid_current_pos','77-3-552,0-3-481964408,3-3-2,2-3-504931,17-3-180,1-3-7'
          'gtid_domain_id','0'
          'gtid_ignore_duplicates','OFF'
          'gtid_seq_no','0'
          'gtid_slave_pos','77-3-552,0-3-481964408,3-3-2,2-3-504931,17-3-180,1-3-7'
          'gtid_strict_mode','OFF'
          'have_compress','YES'
          'have_crypt','YES'
          'have_dynamic_loading','YES'
          'have_geometry','YES'
          'have_openssl','YES'
          'have_profiling','YES'
          'have_query_cache','YES'
          'have_rtree_keys','YES'
          'have_ssl','DISABLED'
          'have_symlink','DISABLED'
          'histogram_size','0'
          'histogram_type','SINGLE_PREC_HB'
          'host_cache_size','128'
          'hostname','optosql5'
          'identity','0'
          'ignore_builtin_innodb','OFF'
          'ignore_db_dirs',''
          'in_transaction','0'
          'init_connect',''
          'init_file',''
          'init_slave',''
          'innodb_adaptive_flushing','ON'
          'innodb_adaptive_flushing_lwm','10.000000'
          'innodb_adaptive_hash_index','ON'
          'innodb_adaptive_hash_index_partitions','1'
          'innodb_adaptive_max_sleep_delay','150000'
          'innodb_additional_mem_pool_size','8388608'
          'innodb_api_bk_commit_interval','5'
          'innodb_api_disable_rowlock','OFF'
          'innodb_api_enable_binlog','OFF'
          'innodb_api_enable_mdl','OFF'
          'innodb_api_trx_level','0'
          'innodb_autoextend_increment','64'
          'innodb_autoinc_lock_mode','1'
          'innodb_buffer_pool_dump_at_shutdown','OFF'
          'innodb_buffer_pool_dump_now','OFF'
          'innodb_buffer_pool_filename','ib_buffer_pool'
          'innodb_buffer_pool_instances','8'
          'innodb_buffer_pool_load_abort','OFF'
          'innodb_buffer_pool_load_at_startup','OFF'
          'innodb_buffer_pool_load_now','OFF'
          'innodb_buffer_pool_populate','OFF'
          'innodb_buffer_pool_size','34359738368'
          'innodb_change_buffer_max_size','25'
          'innodb_change_buffering','all'
          'innodb_checksum_algorithm','innodb'
          'innodb_checksums','ON'
          'innodb_cleaner_lsn_age_factor','high_checkpoint'
          'innodb_cmp_per_index_enabled','OFF'
          'innodb_commit_concurrency','0'
          'innodb_compression_failure_threshold_pct','5'
          'innodb_compression_level','6'
          'innodb_compression_pad_pct_max','50'
          'innodb_concurrency_tickets','5000'
          'innodb_corrupt_table_action','assert'
          'innodb_data_file_path','ibdata1:12M:autoextend'
          'innodb_data_home_dir',''
          'innodb_disable_sort_file_cache','OFF'
          'innodb_doublewrite','ON'
          'innodb_empty_free_list_algorithm','backoff'
          'innodb_fake_changes','OFF'
          'innodb_fast_shutdown','1'
          'innodb_file_format','Antelope'
          'innodb_file_format_check','ON'
          'innodb_file_format_max','Antelope'
          'innodb_file_per_table','ON'
          'innodb_flush_log_at_timeout','1'
          'innodb_flush_log_at_trx_commit','1'
          'innodb_flush_method',''
          'innodb_flush_neighbors','1'
          'innodb_flushing_avg_loops','30'
          'innodb_force_load_corrupted','OFF'
          'innodb_force_recovery','0'
          'innodb_foreground_preflush','exponential_backoff'
          'innodb_ft_aux_table',''
          'innodb_ft_cache_size','8000000'
          'innodb_ft_enable_diag_print','OFF'
          'innodb_ft_enable_stopword','ON'
          'innodb_ft_max_token_size','84'
          'innodb_ft_min_token_size','3'
          'innodb_ft_num_word_optimize','2000'
          'innodb_ft_result_cache_limit','2000000000'
          'innodb_ft_server_stopword_table',''
          'innodb_ft_sort_pll_degree','2'
          'innodb_ft_total_cache_size','640000000'
          'innodb_ft_user_stopword_table',''
          'innodb_io_capacity','200'
          'innodb_io_capacity_max','2000'
          'innodb_kill_idle_transaction','0'
          'innodb_large_prefix','OFF'
          'innodb_lock_wait_timeout','50'
          'innodb_locking_fake_changes','ON'
          'innodb_locks_unsafe_for_binlog','OFF'
          'innodb_log_arch_dir','./'
          'innodb_log_arch_expire_sec','0'
          'innodb_log_archive','OFF'
          'innodb_log_block_size','512'
          'innodb_log_buffer_size','8388608'
          'innodb_log_checksum_algorithm','innodb'
          'innodb_log_compressed_pages','ON'
          'innodb_log_file_size','104857600'
          'innodb_log_files_in_group','2'
          'innodb_log_group_home_dir','./'
          'innodb_lru_scan_depth','1024'
          'innodb_max_bitmap_file_size','104857600'
          'innodb_max_changed_pages','1000000'
          'innodb_max_dirty_pages_pct','75.000000'
          'innodb_max_dirty_pages_pct_lwm','0.001000'
          'innodb_max_purge_lag','0'
          'innodb_max_purge_lag_delay','0'
          'innodb_mirrored_log_groups','1'
          'innodb_monitor_disable',''
          'innodb_monitor_enable',''
          'innodb_monitor_reset',''
          'innodb_monitor_reset_all',''
          'innodb_old_blocks_pct','37'
          'innodb_old_blocks_time','1000'
          'innodb_online_alter_log_max_size','134217728'
          'innodb_open_files','512'
          'innodb_optimize_fulltext_only','OFF'
          'innodb_page_size','16384'
          'innodb_print_all_deadlocks','OFF'
          'innodb_purge_batch_size','300'
          'innodb_purge_threads','1'
          'innodb_random_read_ahead','OFF'
          'innodb_read_ahead_threshold','56'
          'innodb_read_io_threads','64'
          'innodb_read_only','OFF'
          'innodb_replication_delay','0'
          'innodb_rollback_on_timeout','OFF'
          'innodb_rollback_segments','128'
          'innodb_sched_priority_cleaner','19'
          'innodb_show_locks_held','10'
          'innodb_show_verbose_locks','0'
          'innodb_simulate_comp_failures','0'
          'innodb_sort_buffer_size','1048576'
          'innodb_spin_wait_delay','6'
          'innodb_stats_auto_recalc','ON'
          'innodb_stats_method','nulls_equal'
          'innodb_stats_modified_counter','0'
          'innodb_stats_on_metadata','OFF'
          'innodb_stats_persistent','ON'
          'innodb_stats_persistent_sample_pages','20'
          'innodb_stats_sample_pages','8'
          'innodb_stats_traditional','ON'
          'innodb_stats_transient_sample_pages','8'
          'innodb_status_output','OFF'
          'innodb_status_output_locks','OFF'
          'innodb_strict_mode','OFF'
          'innodb_support_xa','ON'
          'innodb_sync_array_size','1'
          'innodb_sync_spin_loops','30'
          'innodb_table_locks','ON'
          'innodb_thread_concurrency','0'
          'innodb_thread_sleep_delay','10000'
          'innodb_track_changed_pages','OFF'
          'innodb_undo_directory','.'
          'innodb_undo_logs','128'
          'innodb_undo_tablespaces','0'
          'innodb_use_atomic_writes','OFF'
          'innodb_use_fallocate','OFF'
          'innodb_use_global_flush_log_at_trx_commit','ON'
          'innodb_use_native_aio','ON'
          'innodb_use_stacktrace','OFF'
          'innodb_use_sys_malloc','ON'
          'innodb_version','5.6.22-71.0'
          'innodb_write_io_threads','64'
          'insert_id','0'
          'interactive_timeout','28800'
          'join_buffer_size','131072'
          'join_buffer_space_limit','2097152'
          'join_cache_level','2'
          'keep_files_on_create','OFF'
          'key_buffer_size','134217728'
          'key_cache_age_threshold','300'
          'key_cache_block_size','1024'
          'key_cache_division_limit','100'
          'key_cache_file_hash_size','512'
          'key_cache_segments','0'
          'large_files_support','ON'
          'large_page_size','0'
          'large_pages','OFF'
          'last_gtid',''
          'last_insert_id','0'
          'lc_messages','en_US'
          'lc_messages_dir',''
          'lc_time_names','en_US'
          'license','GPL'
          'local_infile','ON'
          'lock_wait_timeout','31536000'
          'locked_in_memory','OFF'
          'log_bin','ON'
          'log_bin_trust_function_creators','OFF'
          'log_error','/var/log/mysqld.log'
          'log_output','FILE'
          'log_queries_not_using_indexes','OFF'
          'log_slave_updates','ON'
          'log_slow_filter','admin,filesort,filesort_on_disk,full_join,full_scan,query_cache,query_cache_miss,tmp_table,tmp_table_on_disk'
          'log_slow_rate_limit','1'
          'log_slow_verbosity',''
          'log_warnings','1'
          'long_query_time','10.000000'
          'low_priority_updates','OFF'
          'lower_case_file_system','OFF'
          'lower_case_table_names','1'
          'master_verify_checksum','OFF'
          'max_allowed_packet','33554432'
          'max_binlog_cache_size','18446744073709547520'
          'max_binlog_size','1073741824'
          'max_binlog_stmt_cache_size','18446744073709547520'
          'max_connect_errors','100'
          'max_connections','500'
          'max_delayed_threads','20'
          'max_error_count','64'
          'max_heap_table_size','67108864'
          'max_insert_delayed_threads','20'
          'max_join_size','18446744073709551615'
          'max_length_for_sort_data','1024'
          'max_long_data_size','33554432'
          'max_prepared_stmt_count','16382'
          'max_relay_log_size','1073741824'
          'max_seeks_for_key','4294967295'
          'max_sort_length','1024'
          'max_sp_recursion_depth','0'
          'max_tmp_tables','32'
          'max_user_connections','0'
          'max_write_lock_count','4294967295'
          'metadata_locks_cache_size','1024'
          'metadata_locks_hash_instances','8'
          'min_examined_row_limit','0'
          'mrr_buffer_size','262144'
          'multi_range_count','256'
          'myisam_block_size','1024'
          'myisam_data_pointer_size','6'
          'myisam_max_sort_file_size','9223372036853727232'
          'myisam_mmap_size','18446744073709551615'
          'myisam_recover_options','DEFAULT'
          'myisam_repair_threads','1'
          'myisam_sort_buffer_size','134216704'
          'myisam_stats_method','nulls_unequal'
          'myisam_use_mmap','OFF'
          'net_buffer_length','16384'
          'net_read_timeout','30'
          'net_retry_count','10'
          'net_write_timeout','60'
          'old','OFF'
          'old_alter_table','OFF'
          'old_mode',''
          'old_passwords','OFF'
          'open_files_limit','500005'
          'optimizer_prune_level','1'
          'optimizer_search_depth','62'
          'optimizer_selectivity_sampling_limit','100'
          'optimizer_switch','index_merge=on,index_merge_union=on,index_merge_sort_union=on,index_merge_intersection=on,index_merge_sort_intersection=off,engine_condition_pushdown=off,index_condition_pushdown=on,derived_merge=on,derived_with_keys=on,firstmatch=on,loosescan=on,materialization=on,in_to_exists=on,semijoin=on,partial_match_rowid_merge=on,partial_match_table_scan=on,subquery_cache=on,mrr=off,mrr_cost_based=off,mrr_sort_keys=off,outer_join_with_cache=on,semijoin_with_cache=on,join_cache_incremental=on,join_cache_hashed=on,join_cache_bka=on,optimize_join_buffer_size=off,table_elimination=on,extended_keys=on,exists_to_in=on'
          'optimizer_use_condition_selectivity','1'
          'performance_schema','ON'
          'performance_schema_accounts_size','100'
          'performance_schema_digests_size','10000'
          'performance_schema_events_stages_history_long_size','10000'
          'performance_schema_events_stages_history_size','10'
          'performance_schema_events_statements_history_long_size','10000'
          'performance_schema_events_statements_history_size','10'
          'performance_schema_events_waits_history_long_size','10000'
          'performance_schema_events_waits_history_size','10'
          'performance_schema_hosts_size','100'
          'performance_schema_max_cond_classes','80'
          'performance_schema_max_cond_instances','400900'
          'performance_schema_max_file_classes','50'
          'performance_schema_max_file_handles','32768'
          'performance_schema_max_file_instances','2154'
          'performance_schema_max_mutex_classes','200'
          'performance_schema_max_mutex_instances','605000'
          'performance_schema_max_rwlock_classes','40'
          'performance_schema_max_rwlock_instances','202800'
          'performance_schema_max_socket_classes','10'
          'performance_schema_max_socket_instances','200020'
          'performance_schema_max_stage_classes','150'
          'performance_schema_max_statement_classes','180'
          'performance_schema_max_table_handles','800'
          'performance_schema_max_table_instances','12500'
          'performance_schema_max_thread_classes','50'
          'performance_schema_max_thread_instances','200100'
          'performance_schema_session_connect_attrs_size','512'
          'performance_schema_setup_actors_size','100'
          'performance_schema_setup_objects_size','100'
          'performance_schema_users_size','100'
          'pid_file','/var/lib/mysql/optosql5.pid'
          'plugin_dir','/usr/lib64/mysql/plugin/'
          'plugin_maturity','unknown'
          'port','3306'
          'preload_buffer_size','32768'
          'profiling','OFF'
          'profiling_history_size','15'
          'progress_report_time','5'
          'protocol_version','10'
          'proxy_user',''
          'pseudo_slave_mode','OFF'
          'pseudo_thread_id','189143'
          'query_alloc_block_size','8192'
          'query_cache_limit','1048576'
          'query_cache_min_res_unit','4096'
          'query_cache_size','67108864'
          'query_cache_strip_comments','OFF'
          'query_cache_type','ON'
          'query_cache_wlock_invalidate','OFF'
          'query_prealloc_size','8192'
          'rand_seed1','0'
          'rand_seed2','0'
          'range_alloc_block_size','4096'
          'read_buffer_size','131072'
          'read_only','ON'
          'read_rnd_buffer_size','262144'
          'relay_log',''
          'relay_log_index',''
          'relay_log_info_file','relay-log.info'
          'relay_log_purge','ON'
          'relay_log_recovery','OFF'
          'relay_log_space_limit','0'
          'replicate_annotate_row_events','OFF'
          'replicate_do_db',''
          'replicate_do_table',''
          'replicate_events_marked_for_skip','replicate'
          'replicate_ignore_db',''
          'replicate_ignore_table',''
          'replicate_wild_do_table',''
          'replicate_wild_ignore_table',''
          'report_host',''
          'report_password',''
          'report_port','3306'
          'report_user',''
          'rowid_merge_buff_size','8388608'
          'rpl_recovery_rank','0'
          'secure_auth','OFF'
          'secure_file_priv',''
          'server_id','5'
          'skip_external_locking','ON'
          'skip_name_resolve','ON'
          'skip_networking','OFF'
          'skip_replication','OFF'
          'skip_show_database','OFF'
          'slave_compressed_protocol','OFF'
          'slave_ddl_exec_mode','IDEMPOTENT'
          'slave_domain_parallel_threads','4'
          'slave_exec_mode','STRICT'
          'slave_load_tmpdir','/tmp'
          'slave_max_allowed_packet','1073741824'
          'slave_net_timeout','3600'
          'slave_parallel_max_queued','131072'
          'slave_parallel_threads','12'
          'slave_skip_errors','OFF'
          'slave_sql_verify_checksum','ON'
          'slave_transaction_retries','10'
          'slave_type_conversions',''
          'slow_launch_time','2'
          'slow_query_log','ON'
          'slow_query_log_file','optosql5-slow.log'
          'socket','/var/lib/mysql/mysql.sock'
          'sort_buffer_size','2097152'
          'sql_auto_is_null','OFF'
          'sql_big_selects','ON'
          'sql_buffer_result','OFF'
          'sql_log_bin','ON'
          'sql_log_off','OFF'
          'sql_mode',''
          'sql_notes','ON'
          'sql_quote_show_create','ON'
          'sql_safe_updates','OFF'
          'sql_select_limit','18446744073709551615'
          'sql_slave_skip_counter','0'
          'sql_warnings','OFF'
          'ssl_ca',''
          'ssl_capath',''
          'ssl_cert',''
          'ssl_cipher',''
          'ssl_crl',''
          'ssl_crlpath',''
          'ssl_key',''
          'storage_engine','InnoDB'
          'stored_program_cache','256'
          'sync_binlog','0'
          'sync_frm','ON'
          'sync_master_info','0'
          'sync_relay_log','0'
          'sync_relay_log_info','0'
          'system_time_zone','CEST'
          'table_definition_cache','400'
          'table_open_cache','512'
          'thread_cache_size','4'
          'thread_concurrency','10'
          'thread_handling','one-thread-per-connection'
          'thread_pool_idle_timeout','60'
          'thread_pool_max_threads','500'
          'thread_pool_oversubscribe','3'
          'thread_pool_size','16'
          'thread_pool_stall_limit','500'
          'thread_stack','294912'
          'time_format','%H:%i:%s'
          'time_zone','SYSTEM'
          'timed_mutexes','OFF'
          'timestamp','1434011008.405971'
          'tmp_table_size','134217728'
          'tmpdir','/tmp'
          'transaction_alloc_block_size','8192'
          'transaction_prealloc_size','4096'
          'tx_isolation','REPEATABLE-READ'
          'tx_read_only','OFF'
          'unique_checks','ON'
          'updatable_views_with_limit','YES'
          'use_stat_tables','NEVER'
          'userstat','OFF'
          'version','10.0.16-MariaDB-log'
          'version_comment','MariaDB Server'
          'version_compile_machine','x86_64'
          'version_compile_os','Linux'
          'version_malloc_library','bundled jemalloc'
          'wait_timeout','28800'
          'warning_count','0'
          }}

          The error occurs about once a week during normal operation.

          If you need further information, or if you want us to test something, please let me know.

          Best regards,

          Thomas Mischke
          thomas.mischke Thomas Mischke made changes -
          Description Hello,

          we are having a random "duplicate key" problem with parallel replication. We were trying hard to reproduce it on our test servers as well, but currently it only happens on our live servers.

          The error we get is the following:

          bq. 150604 2:56:41 [ERROR] Slave SQL: Error 'Duplicate entry 'VerifyOrder_0-249' for key 'identry_idqueue'' on query. Default database: 'optovision'. Query: 'insert into `queueentry`(`identry`,`idqueue`,`entrydate`,`removedate`,`info`,`prio`,`reserved`,`ObjectContext`,`OriginHost`,`ReservingHost`,`UnreserveDate`,`ReservationCounter`,`OptimisticLockField`)values('VerifyOrder_0',249,'2015-06-04 02:56:40.687','2015-06-04 03:11:40.671','',0,null,'semaphore','172.16.185.22',null,null,0,0)', Gtid 0-3-454870355, Internal MariaDB error code: 1062
          bq. 150604 2:56:41 [Warning] Slave: Duplicate entry 'VerifyOrder_0-249' for key 'identry_idqueue' Error_code: 1062
          bq. 150604 2:56:41 [ERROR] Error running query, slave SQL thread aborted. Fix the problem, and restart the slave SQL thread with "SLAVE START". We stopped at log 'mysql-bin.007762' position 645226271; GTID position '77-3-552,0-3-454870353,3-3-2,2-3-469046,17-3-180,1-3-7'
          bq. 150604 2:56:41 [Note] Error reading relay log event: slave SQL thread was killed

          It is always the same table that is involved in this issue, and in all cases we analyzed in deep, it was the same value for the unique key to be written.

          The replication is based on one master and one slave, no restrictions on the data to be transferred, and no direct write requests to the slave. Once a day the slave is stopped for a short moment to synchronize the file system and take a backup on LVM level. At the moment we see no connection between this backup and the error (the point of time it happens seems to be independent)

          The table that is involved is defined as follows:

          {{
          CREATE TABLE `queueentry` (
          `OID` INT(11) NOT NULL AUTO_INCREMENT,
          `identry` VARCHAR(100) NOT NULL,
          `entrydate` DATETIME NULL DEFAULT NULL,
          `removedate` DATETIME NULL DEFAULT NULL,
          `info` VARCHAR(8096) NULL DEFAULT NULL,
          `prio` INT(11) NULL DEFAULT NULL,
          `reserved` DATETIME NULL DEFAULT NULL,
          `idqueue` INT(11) NOT NULL,
          `ObjectContext` VARCHAR(200) NULL DEFAULT NULL,
          `OriginHost` VARCHAR(200) NULL DEFAULT NULL,
          `ReservingHost` VARCHAR(200) NULL DEFAULT NULL,
          `UnreserveDate` DATETIME NULL DEFAULT NULL,
          `ReservationCounter` INT(10) UNSIGNED NOT NULL DEFAULT '0',
          `OptimisticLockField` INT(11) NULL DEFAULT NULL,
          PRIMARY KEY (`OID`),
          UNIQUE INDEX `identry_idqueue` (`identry`, `idqueue`),
          INDEX `idqueue` (`idqueue`),
          INDEX `entrydate_prio_reserved_idqueue` (`idqueue`, `reserved`, `prio`, `entrydate`),
          INDEX `removedate` (`removedate`),
          INDEX `entrydate` (`entrydate`),
          INDEX `UnreserveDate_ReservationCounter` (`ReservationCounter`, `UnreserveDate`)
          )
          COLLATE='utf8_general_ci'
          ENGINE=InnoDB
          AUTO_INCREMENT=8164332;
          }}

          We analyzed the binlog of the master in those cases and found the following:
          * Just before this insert statement there are two transactions bound together for parallel replication.
          * Both of these transactions work on the same table as the insert does.
          * The second of these transactions deletes the entry with the unique key, that will be written by the statement showing the duplicate key error.

          That is the following two statements are bound together:
          {{
          insert into `queueentry`(`identry`,`idqueue`,`entrydate`,`removedate`,`info`,`prio`,`reserved`,`ObjectContext`,`OriginHost`,`ReservingHost`,`UnreserveDate`,`ReservationCounter`,`OptimisticLockField`)values('SME_PrintLabel_0',249,'2015-06-04 02:56:40.671','2015-06-04 03:11:40.656','',0,null,'semaphore','172.16.185.21',null,null,0,0)
          delete from `queueentry` where ((`OID` = 6363790) and (`OptimisticLockField` = 0))
          }}

          And then this next transaction crashes:
          {{
          insert into `queueentry`(`identry`,`idqueue`,`entrydate`,`removedate`,`info`,`prio`,`reserved`,`ObjectContext`,`OriginHost`,`ReservingHost`,`UnreserveDate`,`ReservationCounter`,`OptimisticLockField`)values('VerifyOrder_0',249,'2015-06-04 02:56:40.687','2015-06-04 03:11:40.671','',0,null,'semaphore','172.16.185.22',null,null,0,0)
          }}

          Here is the (I hope) most important part of the binlog of the master:

          {{
          #150604 2:56:40 server id 3 end_log_pos 645225814 GTID 0-3-454870353 cid=5620557227
          /*!100001 SET @@session.gtid_seq_no=454870353*//*!*/;
          BEGIN
          /*!*/;
          # at 645225814
          #150604 2:56:40 server id 3 end_log_pos 645225842 Intvar
          SET INSERT_ID=6363805/*!*/;
          # at 645225842
          #150604 2:56:40 server id 3 end_log_pos 645226244 Query thread_id=19732488 exec_time=0 error_code=0
          SET TIMESTAMP=1433379400/*!*/;
          insert into `queueentry`(`identry`,`idqueue`,`entrydate`,`removedate`,`info`,`prio`,`reserved`,`ObjectContext`,`OriginHost`,`ReservingHost`,`UnreserveDate`,`ReservationCounter`,`OptimisticLockField`)values('SME_PrintLabel_0',249,'2015-06-04 02:56:40.671','2015-06-04 03:11:40.656','',0,null,'semaphore','172.16.185.21',null,null,0,0)
          /*!*/;
          # at 645226244
          #150604 2:56:40 server id 3 end_log_pos 645226271 Xid = 5620557224
          COMMIT/*!*/;
          # at 645226271
          #150604 2:56:40 server id 3 end_log_pos 645226311 GTID 0-3-454870354 cid=5620557227
          /*!100001 SET @@session.gtid_seq_no=454870354*//*!*/;
          BEGIN
          /*!*/;
          # at 645226311
          #150604 2:56:40 server id 3 end_log_pos 645226462 Query thread_id=19732489 exec_time=0 error_code=0
          SET TIMESTAMP=1433379400/*!*/;
          delete from `queueentry` where ((`OID` = 6363790) and (`OptimisticLockField` = 0))
          /*!*/;
          # at 645226462
          #150604 2:56:40 server id 3 end_log_pos 645226489 Xid = 5620557228
          COMMIT/*!*/;
          # at 645226489
          #150604 2:56:40 server id 3 end_log_pos 645226527 GTID 0-3-454870355
          /*!100001 SET @@session.gtid_seq_no=454870355*//*!*/;
          BEGIN
          /*!*/;
          # at 645226527
          #150604 2:56:40 server id 3 end_log_pos 645226555 Intvar
          SET INSERT_ID=6363806/*!*/;
          # at 645226555
          #150604 2:56:40 server id 3 end_log_pos 645226954 Query thread_id=19732488 exec_time=0 error_code=0
          SET TIMESTAMP=1433379400/*!*/;
          insert into `queueentry`(`identry`,`idqueue`,`entrydate`,`removedate`,`info`,`prio`,`reserved`,`ObjectContext`,`OriginHost`,`ReservingHost`,`UnreserveDate`,`ReservationCounter`,`OptimisticLockField`)values('VerifyOrder_0',249,'2015-06-04 02:56:40.687','2015-06-04 03:11:40.671','',0,null,'semaphore','172.16.185.22',null,null,0,0)
          /*!*/;
          # at 645226954
          #150604 2:56:40 server id 3 end_log_pos 645226981 Xid = 5620557238
          COMMIT/*!*/;
          # at 645226981
          }}

          The following additional observations we made:
          * The replication error only happens when the slave is at least one second behind the master. So it seems to us that there is somewhere a race condition at work which only happens when the slave is working very fast on the transactions from the master.
          * When this error happens, the command "stop slave" does not work any longer. We have to kill the database process, since some slave transactions are still waiting on each other.
          * After a restart of the database server we examine the table, and we cannot see the malicious key in the table any longer (even before the restart it is not there). We can restart the replication without further actions.
          * When we restart the replication, the slave is normally a few minutes behind the master. While catching up, we normally get the same error again.
          * As far as we can see, it is always the value 'VerifyOrder_0' that is causing the error, even though other values are written to that table as often as that value. We find this interesting, since it is - in alphabetical order - the largest value that is being stored in that table at the moment. I have only a rough understanding about indexes in databases, but I wonder if this could be related to an index reorganization due to the previous insert and delete statements.
          * The error vanishes if we set binlog_commit_wait_count=0 on the master (which is our current workaround).

          The variables on our slave are set as follows:

          {{
          'Variable_name','Value'
          'aria_block_size','8192'
          'aria_checkpoint_interval','30'
          'aria_checkpoint_log_activity','1048576'
          'aria_force_start_after_recovery_failures','0'
          'aria_group_commit','none'
          'aria_group_commit_interval','0'
          'aria_log_file_size','1073741824'
          'aria_log_purge_type','immediate'
          'aria_max_sort_file_size','9223372036853727232'
          'aria_page_checksum','ON'
          'aria_pagecache_age_threshold','300'
          'aria_pagecache_buffer_size','134217728'
          'aria_pagecache_division_limit','100'
          'aria_pagecache_file_hash_size','512'
          'aria_recover','NORMAL'
          'aria_repair_threads','1'
          'aria_sort_buffer_size','268434432'
          'aria_stats_method','nulls_unequal'
          'aria_sync_log_dir','NEWFILE'
          'aria_used_for_temp_tables','ON'
          'auto_increment_increment','1'
          'auto_increment_offset','1'
          'autocommit','ON'
          'automatic_sp_privileges','ON'
          'back_log','150'
          'basedir','/usr'
          'big_tables','OFF'
          'binlog_annotate_row_events','OFF'
          'binlog_cache_size','32768'
          'binlog_checksum','NONE'
          'binlog_commit_wait_count','0'
          'binlog_commit_wait_usec','100000'
          'binlog_direct_non_transactional_updates','OFF'
          'binlog_format','MIXED'
          'binlog_optimize_thread_scheduling','ON'
          'binlog_stmt_cache_size','32768'
          'bulk_insert_buffer_size','8388608'
          'character_set_client','utf8'
          'character_set_connection','utf8'
          'character_set_database','utf8'
          'character_set_filesystem','binary'
          'character_set_results','utf8'
          'character_set_server','utf8'
          'character_set_system','utf8'
          'character_sets_dir','/usr/share/mysql/charsets/'
          'collation_connection','utf8_general_ci'
          'collation_database','utf8_general_ci'
          'collation_server','utf8_unicode_ci'
          'completion_type','NO_CHAIN'
          'concurrent_insert','AUTO'
          'connect_timeout','10'
          'datadir','/var/lib/mysql/'
          'date_format','%Y-%m-%d'
          'datetime_format','%Y-%m-%d %H:%i:%s'
          'deadlock_search_depth_long','15'
          'deadlock_search_depth_short','4'
          'deadlock_timeout_long','50000000'
          'deadlock_timeout_short','10000'
          'debug_no_thread_alarm','OFF'
          'default_master_connection',''
          'default_regex_flags',''
          'default_storage_engine','InnoDB'
          'default_week_format','0'
          'delay_key_write','ON'
          'delayed_insert_limit','100'
          'delayed_insert_timeout','300'
          'delayed_queue_size','1000'
          'div_precision_increment','4'
          'error_count','0'
          'event_scheduler','OFF'
          'expensive_subquery_limit','100'
          'expire_logs_days','5'
          'external_user',''
          'extra_max_connections','1'
          'extra_port','0'
          'flush','OFF'
          'flush_time','0'
          'foreign_key_checks','ON'
          'ft_boolean_syntax','+ -><()~*:""&|'
          'ft_max_word_len','84'
          'ft_min_word_len','4'
          'ft_query_expansion_limit','20'
          'ft_stopword_file','(built-in)'
          'general_log','OFF'
          'general_log_file','optosql5.log'
          'group_concat_max_len','1024'
          'gtid_binlog_pos','0-3-481964408,3-3-2,2-3-504931'
          'gtid_binlog_state','0-5-479038918,0-3-481964408,3-3-2,2-3-504931'
          'gtid_current_pos','77-3-552,0-3-481964408,3-3-2,2-3-504931,17-3-180,1-3-7'
          'gtid_domain_id','0'
          'gtid_ignore_duplicates','OFF'
          'gtid_seq_no','0'
          'gtid_slave_pos','77-3-552,0-3-481964408,3-3-2,2-3-504931,17-3-180,1-3-7'
          'gtid_strict_mode','OFF'
          'have_compress','YES'
          'have_crypt','YES'
          'have_dynamic_loading','YES'
          'have_geometry','YES'
          'have_openssl','YES'
          'have_profiling','YES'
          'have_query_cache','YES'
          'have_rtree_keys','YES'
          'have_ssl','DISABLED'
          'have_symlink','DISABLED'
          'histogram_size','0'
          'histogram_type','SINGLE_PREC_HB'
          'host_cache_size','128'
          'hostname','optosql5'
          'identity','0'
          'ignore_builtin_innodb','OFF'
          'ignore_db_dirs',''
          'in_transaction','0'
          'init_connect',''
          'init_file',''
          'init_slave',''
          'innodb_adaptive_flushing','ON'
          'innodb_adaptive_flushing_lwm','10.000000'
          'innodb_adaptive_hash_index','ON'
          'innodb_adaptive_hash_index_partitions','1'
          'innodb_adaptive_max_sleep_delay','150000'
          'innodb_additional_mem_pool_size','8388608'
          'innodb_api_bk_commit_interval','5'
          'innodb_api_disable_rowlock','OFF'
          'innodb_api_enable_binlog','OFF'
          'innodb_api_enable_mdl','OFF'
          'innodb_api_trx_level','0'
          'innodb_autoextend_increment','64'
          'innodb_autoinc_lock_mode','1'
          'innodb_buffer_pool_dump_at_shutdown','OFF'
          'innodb_buffer_pool_dump_now','OFF'
          'innodb_buffer_pool_filename','ib_buffer_pool'
          'innodb_buffer_pool_instances','8'
          'innodb_buffer_pool_load_abort','OFF'
          'innodb_buffer_pool_load_at_startup','OFF'
          'innodb_buffer_pool_load_now','OFF'
          'innodb_buffer_pool_populate','OFF'
          'innodb_buffer_pool_size','34359738368'
          'innodb_change_buffer_max_size','25'
          'innodb_change_buffering','all'
          'innodb_checksum_algorithm','innodb'
          'innodb_checksums','ON'
          'innodb_cleaner_lsn_age_factor','high_checkpoint'
          'innodb_cmp_per_index_enabled','OFF'
          'innodb_commit_concurrency','0'
          'innodb_compression_failure_threshold_pct','5'
          'innodb_compression_level','6'
          'innodb_compression_pad_pct_max','50'
          'innodb_concurrency_tickets','5000'
          'innodb_corrupt_table_action','assert'
          'innodb_data_file_path','ibdata1:12M:autoextend'
          'innodb_data_home_dir',''
          'innodb_disable_sort_file_cache','OFF'
          'innodb_doublewrite','ON'
          'innodb_empty_free_list_algorithm','backoff'
          'innodb_fake_changes','OFF'
          'innodb_fast_shutdown','1'
          'innodb_file_format','Antelope'
          'innodb_file_format_check','ON'
          'innodb_file_format_max','Antelope'
          'innodb_file_per_table','ON'
          'innodb_flush_log_at_timeout','1'
          'innodb_flush_log_at_trx_commit','1'
          'innodb_flush_method',''
          'innodb_flush_neighbors','1'
          'innodb_flushing_avg_loops','30'
          'innodb_force_load_corrupted','OFF'
          'innodb_force_recovery','0'
          'innodb_foreground_preflush','exponential_backoff'
          'innodb_ft_aux_table',''
          'innodb_ft_cache_size','8000000'
          'innodb_ft_enable_diag_print','OFF'
          'innodb_ft_enable_stopword','ON'
          'innodb_ft_max_token_size','84'
          'innodb_ft_min_token_size','3'
          'innodb_ft_num_word_optimize','2000'
          'innodb_ft_result_cache_limit','2000000000'
          'innodb_ft_server_stopword_table',''
          'innodb_ft_sort_pll_degree','2'
          'innodb_ft_total_cache_size','640000000'
          'innodb_ft_user_stopword_table',''
          'innodb_io_capacity','200'
          'innodb_io_capacity_max','2000'
          'innodb_kill_idle_transaction','0'
          'innodb_large_prefix','OFF'
          'innodb_lock_wait_timeout','50'
          'innodb_locking_fake_changes','ON'
          'innodb_locks_unsafe_for_binlog','OFF'
          'innodb_log_arch_dir','./'
          'innodb_log_arch_expire_sec','0'
          'innodb_log_archive','OFF'
          'innodb_log_block_size','512'
          'innodb_log_buffer_size','8388608'
          'innodb_log_checksum_algorithm','innodb'
          'innodb_log_compressed_pages','ON'
          'innodb_log_file_size','104857600'
          'innodb_log_files_in_group','2'
          'innodb_log_group_home_dir','./'
          'innodb_lru_scan_depth','1024'
          'innodb_max_bitmap_file_size','104857600'
          'innodb_max_changed_pages','1000000'
          'innodb_max_dirty_pages_pct','75.000000'
          'innodb_max_dirty_pages_pct_lwm','0.001000'
          'innodb_max_purge_lag','0'
          'innodb_max_purge_lag_delay','0'
          'innodb_mirrored_log_groups','1'
          'innodb_monitor_disable',''
          'innodb_monitor_enable',''
          'innodb_monitor_reset',''
          'innodb_monitor_reset_all',''
          'innodb_old_blocks_pct','37'
          'innodb_old_blocks_time','1000'
          'innodb_online_alter_log_max_size','134217728'
          'innodb_open_files','512'
          'innodb_optimize_fulltext_only','OFF'
          'innodb_page_size','16384'
          'innodb_print_all_deadlocks','OFF'
          'innodb_purge_batch_size','300'
          'innodb_purge_threads','1'
          'innodb_random_read_ahead','OFF'
          'innodb_read_ahead_threshold','56'
          'innodb_read_io_threads','64'
          'innodb_read_only','OFF'
          'innodb_replication_delay','0'
          'innodb_rollback_on_timeout','OFF'
          'innodb_rollback_segments','128'
          'innodb_sched_priority_cleaner','19'
          'innodb_show_locks_held','10'
          'innodb_show_verbose_locks','0'
          'innodb_simulate_comp_failures','0'
          'innodb_sort_buffer_size','1048576'
          'innodb_spin_wait_delay','6'
          'innodb_stats_auto_recalc','ON'
          'innodb_stats_method','nulls_equal'
          'innodb_stats_modified_counter','0'
          'innodb_stats_on_metadata','OFF'
          'innodb_stats_persistent','ON'
          'innodb_stats_persistent_sample_pages','20'
          'innodb_stats_sample_pages','8'
          'innodb_stats_traditional','ON'
          'innodb_stats_transient_sample_pages','8'
          'innodb_status_output','OFF'
          'innodb_status_output_locks','OFF'
          'innodb_strict_mode','OFF'
          'innodb_support_xa','ON'
          'innodb_sync_array_size','1'
          'innodb_sync_spin_loops','30'
          'innodb_table_locks','ON'
          'innodb_thread_concurrency','0'
          'innodb_thread_sleep_delay','10000'
          'innodb_track_changed_pages','OFF'
          'innodb_undo_directory','.'
          'innodb_undo_logs','128'
          'innodb_undo_tablespaces','0'
          'innodb_use_atomic_writes','OFF'
          'innodb_use_fallocate','OFF'
          'innodb_use_global_flush_log_at_trx_commit','ON'
          'innodb_use_native_aio','ON'
          'innodb_use_stacktrace','OFF'
          'innodb_use_sys_malloc','ON'
          'innodb_version','5.6.22-71.0'
          'innodb_write_io_threads','64'
          'insert_id','0'
          'interactive_timeout','28800'
          'join_buffer_size','131072'
          'join_buffer_space_limit','2097152'
          'join_cache_level','2'
          'keep_files_on_create','OFF'
          'key_buffer_size','134217728'
          'key_cache_age_threshold','300'
          'key_cache_block_size','1024'
          'key_cache_division_limit','100'
          'key_cache_file_hash_size','512'
          'key_cache_segments','0'
          'large_files_support','ON'
          'large_page_size','0'
          'large_pages','OFF'
          'last_gtid',''
          'last_insert_id','0'
          'lc_messages','en_US'
          'lc_messages_dir',''
          'lc_time_names','en_US'
          'license','GPL'
          'local_infile','ON'
          'lock_wait_timeout','31536000'
          'locked_in_memory','OFF'
          'log_bin','ON'
          'log_bin_trust_function_creators','OFF'
          'log_error','/var/log/mysqld.log'
          'log_output','FILE'
          'log_queries_not_using_indexes','OFF'
          'log_slave_updates','ON'
          'log_slow_filter','admin,filesort,filesort_on_disk,full_join,full_scan,query_cache,query_cache_miss,tmp_table,tmp_table_on_disk'
          'log_slow_rate_limit','1'
          'log_slow_verbosity',''
          'log_warnings','1'
          'long_query_time','10.000000'
          'low_priority_updates','OFF'
          'lower_case_file_system','OFF'
          'lower_case_table_names','1'
          'master_verify_checksum','OFF'
          'max_allowed_packet','33554432'
          'max_binlog_cache_size','18446744073709547520'
          'max_binlog_size','1073741824'
          'max_binlog_stmt_cache_size','18446744073709547520'
          'max_connect_errors','100'
          'max_connections','500'
          'max_delayed_threads','20'
          'max_error_count','64'
          'max_heap_table_size','67108864'
          'max_insert_delayed_threads','20'
          'max_join_size','18446744073709551615'
          'max_length_for_sort_data','1024'
          'max_long_data_size','33554432'
          'max_prepared_stmt_count','16382'
          'max_relay_log_size','1073741824'
          'max_seeks_for_key','4294967295'
          'max_sort_length','1024'
          'max_sp_recursion_depth','0'
          'max_tmp_tables','32'
          'max_user_connections','0'
          'max_write_lock_count','4294967295'
          'metadata_locks_cache_size','1024'
          'metadata_locks_hash_instances','8'
          'min_examined_row_limit','0'
          'mrr_buffer_size','262144'
          'multi_range_count','256'
          'myisam_block_size','1024'
          'myisam_data_pointer_size','6'
          'myisam_max_sort_file_size','9223372036853727232'
          'myisam_mmap_size','18446744073709551615'
          'myisam_recover_options','DEFAULT'
          'myisam_repair_threads','1'
          'myisam_sort_buffer_size','134216704'
          'myisam_stats_method','nulls_unequal'
          'myisam_use_mmap','OFF'
          'net_buffer_length','16384'
          'net_read_timeout','30'
          'net_retry_count','10'
          'net_write_timeout','60'
          'old','OFF'
          'old_alter_table','OFF'
          'old_mode',''
          'old_passwords','OFF'
          'open_files_limit','500005'
          'optimizer_prune_level','1'
          'optimizer_search_depth','62'
          'optimizer_selectivity_sampling_limit','100'
          'optimizer_switch','index_merge=on,index_merge_union=on,index_merge_sort_union=on,index_merge_intersection=on,index_merge_sort_intersection=off,engine_condition_pushdown=off,index_condition_pushdown=on,derived_merge=on,derived_with_keys=on,firstmatch=on,loosescan=on,materialization=on,in_to_exists=on,semijoin=on,partial_match_rowid_merge=on,partial_match_table_scan=on,subquery_cache=on,mrr=off,mrr_cost_based=off,mrr_sort_keys=off,outer_join_with_cache=on,semijoin_with_cache=on,join_cache_incremental=on,join_cache_hashed=on,join_cache_bka=on,optimize_join_buffer_size=off,table_elimination=on,extended_keys=on,exists_to_in=on'
          'optimizer_use_condition_selectivity','1'
          'performance_schema','ON'
          'performance_schema_accounts_size','100'
          'performance_schema_digests_size','10000'
          'performance_schema_events_stages_history_long_size','10000'
          'performance_schema_events_stages_history_size','10'
          'performance_schema_events_statements_history_long_size','10000'
          'performance_schema_events_statements_history_size','10'
          'performance_schema_events_waits_history_long_size','10000'
          'performance_schema_events_waits_history_size','10'
          'performance_schema_hosts_size','100'
          'performance_schema_max_cond_classes','80'
          'performance_schema_max_cond_instances','400900'
          'performance_schema_max_file_classes','50'
          'performance_schema_max_file_handles','32768'
          'performance_schema_max_file_instances','2154'
          'performance_schema_max_mutex_classes','200'
          'performance_schema_max_mutex_instances','605000'
          'performance_schema_max_rwlock_classes','40'
          'performance_schema_max_rwlock_instances','202800'
          'performance_schema_max_socket_classes','10'
          'performance_schema_max_socket_instances','200020'
          'performance_schema_max_stage_classes','150'
          'performance_schema_max_statement_classes','180'
          'performance_schema_max_table_handles','800'
          'performance_schema_max_table_instances','12500'
          'performance_schema_max_thread_classes','50'
          'performance_schema_max_thread_instances','200100'
          'performance_schema_session_connect_attrs_size','512'
          'performance_schema_setup_actors_size','100'
          'performance_schema_setup_objects_size','100'
          'performance_schema_users_size','100'
          'pid_file','/var/lib/mysql/optosql5.pid'
          'plugin_dir','/usr/lib64/mysql/plugin/'
          'plugin_maturity','unknown'
          'port','3306'
          'preload_buffer_size','32768'
          'profiling','OFF'
          'profiling_history_size','15'
          'progress_report_time','5'
          'protocol_version','10'
          'proxy_user',''
          'pseudo_slave_mode','OFF'
          'pseudo_thread_id','189143'
          'query_alloc_block_size','8192'
          'query_cache_limit','1048576'
          'query_cache_min_res_unit','4096'
          'query_cache_size','67108864'
          'query_cache_strip_comments','OFF'
          'query_cache_type','ON'
          'query_cache_wlock_invalidate','OFF'
          'query_prealloc_size','8192'
          'rand_seed1','0'
          'rand_seed2','0'
          'range_alloc_block_size','4096'
          'read_buffer_size','131072'
          'read_only','ON'
          'read_rnd_buffer_size','262144'
          'relay_log',''
          'relay_log_index',''
          'relay_log_info_file','relay-log.info'
          'relay_log_purge','ON'
          'relay_log_recovery','OFF'
          'relay_log_space_limit','0'
          'replicate_annotate_row_events','OFF'
          'replicate_do_db',''
          'replicate_do_table',''
          'replicate_events_marked_for_skip','replicate'
          'replicate_ignore_db',''
          'replicate_ignore_table',''
          'replicate_wild_do_table',''
          'replicate_wild_ignore_table',''
          'report_host',''
          'report_password',''
          'report_port','3306'
          'report_user',''
          'rowid_merge_buff_size','8388608'
          'rpl_recovery_rank','0'
          'secure_auth','OFF'
          'secure_file_priv',''
          'server_id','5'
          'skip_external_locking','ON'
          'skip_name_resolve','ON'
          'skip_networking','OFF'
          'skip_replication','OFF'
          'skip_show_database','OFF'
          'slave_compressed_protocol','OFF'
          'slave_ddl_exec_mode','IDEMPOTENT'
          'slave_domain_parallel_threads','4'
          'slave_exec_mode','STRICT'
          'slave_load_tmpdir','/tmp'
          'slave_max_allowed_packet','1073741824'
          'slave_net_timeout','3600'
          'slave_parallel_max_queued','131072'
          'slave_parallel_threads','12'
          'slave_skip_errors','OFF'
          'slave_sql_verify_checksum','ON'
          'slave_transaction_retries','10'
          'slave_type_conversions',''
          'slow_launch_time','2'
          'slow_query_log','ON'
          'slow_query_log_file','optosql5-slow.log'
          'socket','/var/lib/mysql/mysql.sock'
          'sort_buffer_size','2097152'
          'sql_auto_is_null','OFF'
          'sql_big_selects','ON'
          'sql_buffer_result','OFF'
          'sql_log_bin','ON'
          'sql_log_off','OFF'
          'sql_mode',''
          'sql_notes','ON'
          'sql_quote_show_create','ON'
          'sql_safe_updates','OFF'
          'sql_select_limit','18446744073709551615'
          'sql_slave_skip_counter','0'
          'sql_warnings','OFF'
          'ssl_ca',''
          'ssl_capath',''
          'ssl_cert',''
          'ssl_cipher',''
          'ssl_crl',''
          'ssl_crlpath',''
          'ssl_key',''
          'storage_engine','InnoDB'
          'stored_program_cache','256'
          'sync_binlog','0'
          'sync_frm','ON'
          'sync_master_info','0'
          'sync_relay_log','0'
          'sync_relay_log_info','0'
          'system_time_zone','CEST'
          'table_definition_cache','400'
          'table_open_cache','512'
          'thread_cache_size','4'
          'thread_concurrency','10'
          'thread_handling','one-thread-per-connection'
          'thread_pool_idle_timeout','60'
          'thread_pool_max_threads','500'
          'thread_pool_oversubscribe','3'
          'thread_pool_size','16'
          'thread_pool_stall_limit','500'
          'thread_stack','294912'
          'time_format','%H:%i:%s'
          'time_zone','SYSTEM'
          'timed_mutexes','OFF'
          'timestamp','1434011008.405971'
          'tmp_table_size','134217728'
          'tmpdir','/tmp'
          'transaction_alloc_block_size','8192'
          'transaction_prealloc_size','4096'
          'tx_isolation','REPEATABLE-READ'
          'tx_read_only','OFF'
          'unique_checks','ON'
          'updatable_views_with_limit','YES'
          'use_stat_tables','NEVER'
          'userstat','OFF'
          'version','10.0.16-MariaDB-log'
          'version_comment','MariaDB Server'
          'version_compile_machine','x86_64'
          'version_compile_os','Linux'
          'version_malloc_library','bundled jemalloc'
          'wait_timeout','28800'
          'warning_count','0'
          }}

          The error occurs about once a week during normal operation.

          If you need further information, or if you want us to test something, please let me know.

          Best regards,

          Thomas Mischke
          Hello,

          we are having a random "duplicate key" problem with parallel replication. We were trying hard to reproduce it on our test servers as well, but currently it only happens on our live servers.

          The error we get is the following:

          150604 2:56:41 [ERROR] Slave SQL: Error 'Duplicate entry 'VerifyOrder_0-249' for key 'identry_idqueue'' on query. Default database: 'optovision'. Query: 'insert into `queueentry`(`identry`,`idqueue`,`entrydate`,`removedate`,`info`,`prio`,`reserved`,`ObjectContext`,`OriginHost`,`ReservingHost`,`UnreserveDate`,`ReservationCounter`,`OptimisticLockField`)values('VerifyOrder_0',249,'2015-06-04 02:56:40.687','2015-06-04 03:11:40.671','',0,null,'semaphore','172.16.185.22',null,null,0,0)', Gtid 0-3-454870355, Internal MariaDB error code: 1062
          150604 2:56:41 [Warning] Slave: Duplicate entry 'VerifyOrder_0-249' for key 'identry_idqueue' Error_code: 1062
          150604 2:56:41 [ERROR] Error running query, slave SQL thread aborted. Fix the problem, and restart the slave SQL thread with "SLAVE START". We stopped at log 'mysql-bin.007762' position 645226271; GTID position '77-3-552,0-3-454870353,3-3-2,2-3-469046,17-3-180,1-3-7'
          150604 2:56:41 [Note] Error reading relay log event: slave SQL thread was killed

          It is always the same table that is involved in this issue, and in all cases we analyzed in deep, it was the same value for the unique key to be written.

          The replication is based on one master and one slave, no restrictions on the data to be transferred, and no direct write requests to the slave. Once a day the slave is stopped for a short moment to synchronize the file system and take a backup on LVM level. At the moment we see no connection between this backup and the error (the point of time it happens seems to be independent)

          The table that is involved is defined as follows:

          {{
          CREATE TABLE `queueentry` (
          `OID` INT(11) NOT NULL AUTO_INCREMENT,
          `identry` VARCHAR(100) NOT NULL,
          `entrydate` DATETIME NULL DEFAULT NULL,
          `removedate` DATETIME NULL DEFAULT NULL,
          `info` VARCHAR(8096) NULL DEFAULT NULL,
          `prio` INT(11) NULL DEFAULT NULL,
          `reserved` DATETIME NULL DEFAULT NULL,
          `idqueue` INT(11) NOT NULL,
          `ObjectContext` VARCHAR(200) NULL DEFAULT NULL,
          `OriginHost` VARCHAR(200) NULL DEFAULT NULL,
          `ReservingHost` VARCHAR(200) NULL DEFAULT NULL,
          `UnreserveDate` DATETIME NULL DEFAULT NULL,
          `ReservationCounter` INT(10) UNSIGNED NOT NULL DEFAULT '0',
          `OptimisticLockField` INT(11) NULL DEFAULT NULL,
          PRIMARY KEY (`OID`),
          UNIQUE INDEX `identry_idqueue` (`identry`, `idqueue`),
          INDEX `idqueue` (`idqueue`),
          INDEX `entrydate_prio_reserved_idqueue` (`idqueue`, `reserved`, `prio`, `entrydate`),
          INDEX `removedate` (`removedate`),
          INDEX `entrydate` (`entrydate`),
          INDEX `UnreserveDate_ReservationCounter` (`ReservationCounter`, `UnreserveDate`)
          )
          COLLATE='utf8_general_ci'
          ENGINE=InnoDB
          AUTO_INCREMENT=8164332;
          }}

          We analyzed the binlog of the master in those cases and found the following:
          * Just before this insert statement there are two transactions bound together for parallel replication.
          * Both of these transactions work on the same table as the insert does.
          * The second of these transactions deletes the entry with the unique key, that will be written by the statement showing the duplicate key error.

          That is the following two statements are bound together:
          {{
          insert into `queueentry`(`identry`,`idqueue`,`entrydate`,`removedate`,`info`,`prio`,`reserved`,`ObjectContext`,`OriginHost`,`ReservingHost`,`UnreserveDate`,`ReservationCounter`,`OptimisticLockField`)values('SME_PrintLabel_0',249,'2015-06-04 02:56:40.671','2015-06-04 03:11:40.656','',0,null,'semaphore','172.16.185.21',null,null,0,0)
          delete from `queueentry` where ((`OID` = 6363790) and (`OptimisticLockField` = 0))
          }}

          And then this next transaction crashes:
          {{
          insert into `queueentry`(`identry`,`idqueue`,`entrydate`,`removedate`,`info`,`prio`,`reserved`,`ObjectContext`,`OriginHost`,`ReservingHost`,`UnreserveDate`,`ReservationCounter`,`OptimisticLockField`)values('VerifyOrder_0',249,'2015-06-04 02:56:40.687','2015-06-04 03:11:40.671','',0,null,'semaphore','172.16.185.22',null,null,0,0)
          }}

          Here is the (I hope) most important part of the binlog of the master:

          {{
          #150604 2:56:40 server id 3 end_log_pos 645225814 GTID 0-3-454870353 cid=5620557227
          /*!100001 SET @@session.gtid_seq_no=454870353*//*!*/;
          BEGIN
          /*!*/;
          # at 645225814
          #150604 2:56:40 server id 3 end_log_pos 645225842 Intvar
          SET INSERT_ID=6363805/*!*/;
          # at 645225842
          #150604 2:56:40 server id 3 end_log_pos 645226244 Query thread_id=19732488 exec_time=0 error_code=0
          SET TIMESTAMP=1433379400/*!*/;
          insert into `queueentry`(`identry`,`idqueue`,`entrydate`,`removedate`,`info`,`prio`,`reserved`,`ObjectContext`,`OriginHost`,`ReservingHost`,`UnreserveDate`,`ReservationCounter`,`OptimisticLockField`)values('SME_PrintLabel_0',249,'2015-06-04 02:56:40.671','2015-06-04 03:11:40.656','',0,null,'semaphore','172.16.185.21',null,null,0,0)
          /*!*/;
          # at 645226244
          #150604 2:56:40 server id 3 end_log_pos 645226271 Xid = 5620557224
          COMMIT/*!*/;
          # at 645226271
          #150604 2:56:40 server id 3 end_log_pos 645226311 GTID 0-3-454870354 cid=5620557227
          /*!100001 SET @@session.gtid_seq_no=454870354*//*!*/;
          BEGIN
          /*!*/;
          # at 645226311
          #150604 2:56:40 server id 3 end_log_pos 645226462 Query thread_id=19732489 exec_time=0 error_code=0
          SET TIMESTAMP=1433379400/*!*/;
          delete from `queueentry` where ((`OID` = 6363790) and (`OptimisticLockField` = 0))
          /*!*/;
          # at 645226462
          #150604 2:56:40 server id 3 end_log_pos 645226489 Xid = 5620557228
          COMMIT/*!*/;
          # at 645226489
          #150604 2:56:40 server id 3 end_log_pos 645226527 GTID 0-3-454870355
          /*!100001 SET @@session.gtid_seq_no=454870355*//*!*/;
          BEGIN
          /*!*/;
          # at 645226527
          #150604 2:56:40 server id 3 end_log_pos 645226555 Intvar
          SET INSERT_ID=6363806/*!*/;
          # at 645226555
          #150604 2:56:40 server id 3 end_log_pos 645226954 Query thread_id=19732488 exec_time=0 error_code=0
          SET TIMESTAMP=1433379400/*!*/;
          insert into `queueentry`(`identry`,`idqueue`,`entrydate`,`removedate`,`info`,`prio`,`reserved`,`ObjectContext`,`OriginHost`,`ReservingHost`,`UnreserveDate`,`ReservationCounter`,`OptimisticLockField`)values('VerifyOrder_0',249,'2015-06-04 02:56:40.687','2015-06-04 03:11:40.671','',0,null,'semaphore','172.16.185.22',null,null,0,0)
          /*!*/;
          # at 645226954
          #150604 2:56:40 server id 3 end_log_pos 645226981 Xid = 5620557238
          COMMIT/*!*/;
          # at 645226981
          }}

          The following additional observations we made:
          * The replication error only happens when the slave is at least one second behind the master. So it seems to us that there is somewhere a race condition at work which only happens when the slave is working very fast on the transactions from the master.
          * When this error happens, the command "stop slave" does not work any longer. We have to kill the database process, since some slave transactions are still waiting on each other.
          * After a restart of the database server we examine the table, and we cannot see the malicious key in the table any longer (even before the restart it is not there). We can restart the replication without further actions.
          * When we restart the replication, the slave is normally a few minutes behind the master. While catching up, we normally get the same error again.
          * As far as we can see, it is always the value 'VerifyOrder_0' that is causing the error, even though other values are written to that table as often as that value. We find this interesting, since it is - in alphabetical order - the largest value that is being stored in that table at the moment. I have only a rough understanding about indexes in databases, but I wonder if this could be related to an index reorganization due to the previous insert and delete statements.
          * The error vanishes if we set binlog_commit_wait_count=0 on the master (which is our current workaround).

          The variables on our slave are set as follows:

          {{
          'Variable_name','Value'
          'aria_block_size','8192'
          'aria_checkpoint_interval','30'
          'aria_checkpoint_log_activity','1048576'
          'aria_force_start_after_recovery_failures','0'
          'aria_group_commit','none'
          'aria_group_commit_interval','0'
          'aria_log_file_size','1073741824'
          'aria_log_purge_type','immediate'
          'aria_max_sort_file_size','9223372036853727232'
          'aria_page_checksum','ON'
          'aria_pagecache_age_threshold','300'
          'aria_pagecache_buffer_size','134217728'
          'aria_pagecache_division_limit','100'
          'aria_pagecache_file_hash_size','512'
          'aria_recover','NORMAL'
          'aria_repair_threads','1'
          'aria_sort_buffer_size','268434432'
          'aria_stats_method','nulls_unequal'
          'aria_sync_log_dir','NEWFILE'
          'aria_used_for_temp_tables','ON'
          'auto_increment_increment','1'
          'auto_increment_offset','1'
          'autocommit','ON'
          'automatic_sp_privileges','ON'
          'back_log','150'
          'basedir','/usr'
          'big_tables','OFF'
          'binlog_annotate_row_events','OFF'
          'binlog_cache_size','32768'
          'binlog_checksum','NONE'
          'binlog_commit_wait_count','0'
          'binlog_commit_wait_usec','100000'
          'binlog_direct_non_transactional_updates','OFF'
          'binlog_format','MIXED'
          'binlog_optimize_thread_scheduling','ON'
          'binlog_stmt_cache_size','32768'
          'bulk_insert_buffer_size','8388608'
          'character_set_client','utf8'
          'character_set_connection','utf8'
          'character_set_database','utf8'
          'character_set_filesystem','binary'
          'character_set_results','utf8'
          'character_set_server','utf8'
          'character_set_system','utf8'
          'character_sets_dir','/usr/share/mysql/charsets/'
          'collation_connection','utf8_general_ci'
          'collation_database','utf8_general_ci'
          'collation_server','utf8_unicode_ci'
          'completion_type','NO_CHAIN'
          'concurrent_insert','AUTO'
          'connect_timeout','10'
          'datadir','/var/lib/mysql/'
          'date_format','%Y-%m-%d'
          'datetime_format','%Y-%m-%d %H:%i:%s'
          'deadlock_search_depth_long','15'
          'deadlock_search_depth_short','4'
          'deadlock_timeout_long','50000000'
          'deadlock_timeout_short','10000'
          'debug_no_thread_alarm','OFF'
          'default_master_connection',''
          'default_regex_flags',''
          'default_storage_engine','InnoDB'
          'default_week_format','0'
          'delay_key_write','ON'
          'delayed_insert_limit','100'
          'delayed_insert_timeout','300'
          'delayed_queue_size','1000'
          'div_precision_increment','4'
          'error_count','0'
          'event_scheduler','OFF'
          'expensive_subquery_limit','100'
          'expire_logs_days','5'
          'external_user',''
          'extra_max_connections','1'
          'extra_port','0'
          'flush','OFF'
          'flush_time','0'
          'foreign_key_checks','ON'
          'ft_boolean_syntax','+ -><()~*:""&|'
          'ft_max_word_len','84'
          'ft_min_word_len','4'
          'ft_query_expansion_limit','20'
          'ft_stopword_file','(built-in)'
          'general_log','OFF'
          'general_log_file','optosql5.log'
          'group_concat_max_len','1024'
          'gtid_binlog_pos','0-3-481964408,3-3-2,2-3-504931'
          'gtid_binlog_state','0-5-479038918,0-3-481964408,3-3-2,2-3-504931'
          'gtid_current_pos','77-3-552,0-3-481964408,3-3-2,2-3-504931,17-3-180,1-3-7'
          'gtid_domain_id','0'
          'gtid_ignore_duplicates','OFF'
          'gtid_seq_no','0'
          'gtid_slave_pos','77-3-552,0-3-481964408,3-3-2,2-3-504931,17-3-180,1-3-7'
          'gtid_strict_mode','OFF'
          'have_compress','YES'
          'have_crypt','YES'
          'have_dynamic_loading','YES'
          'have_geometry','YES'
          'have_openssl','YES'
          'have_profiling','YES'
          'have_query_cache','YES'
          'have_rtree_keys','YES'
          'have_ssl','DISABLED'
          'have_symlink','DISABLED'
          'histogram_size','0'
          'histogram_type','SINGLE_PREC_HB'
          'host_cache_size','128'
          'hostname','optosql5'
          'identity','0'
          'ignore_builtin_innodb','OFF'
          'ignore_db_dirs',''
          'in_transaction','0'
          'init_connect',''
          'init_file',''
          'init_slave',''
          'innodb_adaptive_flushing','ON'
          'innodb_adaptive_flushing_lwm','10.000000'
          'innodb_adaptive_hash_index','ON'
          'innodb_adaptive_hash_index_partitions','1'
          'innodb_adaptive_max_sleep_delay','150000'
          'innodb_additional_mem_pool_size','8388608'
          'innodb_api_bk_commit_interval','5'
          'innodb_api_disable_rowlock','OFF'
          'innodb_api_enable_binlog','OFF'
          'innodb_api_enable_mdl','OFF'
          'innodb_api_trx_level','0'
          'innodb_autoextend_increment','64'
          'innodb_autoinc_lock_mode','1'
          'innodb_buffer_pool_dump_at_shutdown','OFF'
          'innodb_buffer_pool_dump_now','OFF'
          'innodb_buffer_pool_filename','ib_buffer_pool'
          'innodb_buffer_pool_instances','8'
          'innodb_buffer_pool_load_abort','OFF'
          'innodb_buffer_pool_load_at_startup','OFF'
          'innodb_buffer_pool_load_now','OFF'
          'innodb_buffer_pool_populate','OFF'
          'innodb_buffer_pool_size','34359738368'
          'innodb_change_buffer_max_size','25'
          'innodb_change_buffering','all'
          'innodb_checksum_algorithm','innodb'
          'innodb_checksums','ON'
          'innodb_cleaner_lsn_age_factor','high_checkpoint'
          'innodb_cmp_per_index_enabled','OFF'
          'innodb_commit_concurrency','0'
          'innodb_compression_failure_threshold_pct','5'
          'innodb_compression_level','6'
          'innodb_compression_pad_pct_max','50'
          'innodb_concurrency_tickets','5000'
          'innodb_corrupt_table_action','assert'
          'innodb_data_file_path','ibdata1:12M:autoextend'
          'innodb_data_home_dir',''
          'innodb_disable_sort_file_cache','OFF'
          'innodb_doublewrite','ON'
          'innodb_empty_free_list_algorithm','backoff'
          'innodb_fake_changes','OFF'
          'innodb_fast_shutdown','1'
          'innodb_file_format','Antelope'
          'innodb_file_format_check','ON'
          'innodb_file_format_max','Antelope'
          'innodb_file_per_table','ON'
          'innodb_flush_log_at_timeout','1'
          'innodb_flush_log_at_trx_commit','1'
          'innodb_flush_method',''
          'innodb_flush_neighbors','1'
          'innodb_flushing_avg_loops','30'
          'innodb_force_load_corrupted','OFF'
          'innodb_force_recovery','0'
          'innodb_foreground_preflush','exponential_backoff'
          'innodb_ft_aux_table',''
          'innodb_ft_cache_size','8000000'
          'innodb_ft_enable_diag_print','OFF'
          'innodb_ft_enable_stopword','ON'
          'innodb_ft_max_token_size','84'
          'innodb_ft_min_token_size','3'
          'innodb_ft_num_word_optimize','2000'
          'innodb_ft_result_cache_limit','2000000000'
          'innodb_ft_server_stopword_table',''
          'innodb_ft_sort_pll_degree','2'
          'innodb_ft_total_cache_size','640000000'
          'innodb_ft_user_stopword_table',''
          'innodb_io_capacity','200'
          'innodb_io_capacity_max','2000'
          'innodb_kill_idle_transaction','0'
          'innodb_large_prefix','OFF'
          'innodb_lock_wait_timeout','50'
          'innodb_locking_fake_changes','ON'
          'innodb_locks_unsafe_for_binlog','OFF'
          'innodb_log_arch_dir','./'
          'innodb_log_arch_expire_sec','0'
          'innodb_log_archive','OFF'
          'innodb_log_block_size','512'
          'innodb_log_buffer_size','8388608'
          'innodb_log_checksum_algorithm','innodb'
          'innodb_log_compressed_pages','ON'
          'innodb_log_file_size','104857600'
          'innodb_log_files_in_group','2'
          'innodb_log_group_home_dir','./'
          'innodb_lru_scan_depth','1024'
          'innodb_max_bitmap_file_size','104857600'
          'innodb_max_changed_pages','1000000'
          'innodb_max_dirty_pages_pct','75.000000'
          'innodb_max_dirty_pages_pct_lwm','0.001000'
          'innodb_max_purge_lag','0'
          'innodb_max_purge_lag_delay','0'
          'innodb_mirrored_log_groups','1'
          'innodb_monitor_disable',''
          'innodb_monitor_enable',''
          'innodb_monitor_reset',''
          'innodb_monitor_reset_all',''
          'innodb_old_blocks_pct','37'
          'innodb_old_blocks_time','1000'
          'innodb_online_alter_log_max_size','134217728'
          'innodb_open_files','512'
          'innodb_optimize_fulltext_only','OFF'
          'innodb_page_size','16384'
          'innodb_print_all_deadlocks','OFF'
          'innodb_purge_batch_size','300'
          'innodb_purge_threads','1'
          'innodb_random_read_ahead','OFF'
          'innodb_read_ahead_threshold','56'
          'innodb_read_io_threads','64'
          'innodb_read_only','OFF'
          'innodb_replication_delay','0'
          'innodb_rollback_on_timeout','OFF'
          'innodb_rollback_segments','128'
          'innodb_sched_priority_cleaner','19'
          'innodb_show_locks_held','10'
          'innodb_show_verbose_locks','0'
          'innodb_simulate_comp_failures','0'
          'innodb_sort_buffer_size','1048576'
          'innodb_spin_wait_delay','6'
          'innodb_stats_auto_recalc','ON'
          'innodb_stats_method','nulls_equal'
          'innodb_stats_modified_counter','0'
          'innodb_stats_on_metadata','OFF'
          'innodb_stats_persistent','ON'
          'innodb_stats_persistent_sample_pages','20'
          'innodb_stats_sample_pages','8'
          'innodb_stats_traditional','ON'
          'innodb_stats_transient_sample_pages','8'
          'innodb_status_output','OFF'
          'innodb_status_output_locks','OFF'
          'innodb_strict_mode','OFF'
          'innodb_support_xa','ON'
          'innodb_sync_array_size','1'
          'innodb_sync_spin_loops','30'
          'innodb_table_locks','ON'
          'innodb_thread_concurrency','0'
          'innodb_thread_sleep_delay','10000'
          'innodb_track_changed_pages','OFF'
          'innodb_undo_directory','.'
          'innodb_undo_logs','128'
          'innodb_undo_tablespaces','0'
          'innodb_use_atomic_writes','OFF'
          'innodb_use_fallocate','OFF'
          'innodb_use_global_flush_log_at_trx_commit','ON'
          'innodb_use_native_aio','ON'
          'innodb_use_stacktrace','OFF'
          'innodb_use_sys_malloc','ON'
          'innodb_version','5.6.22-71.0'
          'innodb_write_io_threads','64'
          'insert_id','0'
          'interactive_timeout','28800'
          'join_buffer_size','131072'
          'join_buffer_space_limit','2097152'
          'join_cache_level','2'
          'keep_files_on_create','OFF'
          'key_buffer_size','134217728'
          'key_cache_age_threshold','300'
          'key_cache_block_size','1024'
          'key_cache_division_limit','100'
          'key_cache_file_hash_size','512'
          'key_cache_segments','0'
          'large_files_support','ON'
          'large_page_size','0'
          'large_pages','OFF'
          'last_gtid',''
          'last_insert_id','0'
          'lc_messages','en_US'
          'lc_messages_dir',''
          'lc_time_names','en_US'
          'license','GPL'
          'local_infile','ON'
          'lock_wait_timeout','31536000'
          'locked_in_memory','OFF'
          'log_bin','ON'
          'log_bin_trust_function_creators','OFF'
          'log_error','/var/log/mysqld.log'
          'log_output','FILE'
          'log_queries_not_using_indexes','OFF'
          'log_slave_updates','ON'
          'log_slow_filter','admin,filesort,filesort_on_disk,full_join,full_scan,query_cache,query_cache_miss,tmp_table,tmp_table_on_disk'
          'log_slow_rate_limit','1'
          'log_slow_verbosity',''
          'log_warnings','1'
          'long_query_time','10.000000'
          'low_priority_updates','OFF'
          'lower_case_file_system','OFF'
          'lower_case_table_names','1'
          'master_verify_checksum','OFF'
          'max_allowed_packet','33554432'
          'max_binlog_cache_size','18446744073709547520'
          'max_binlog_size','1073741824'
          'max_binlog_stmt_cache_size','18446744073709547520'
          'max_connect_errors','100'
          'max_connections','500'
          'max_delayed_threads','20'
          'max_error_count','64'
          'max_heap_table_size','67108864'
          'max_insert_delayed_threads','20'
          'max_join_size','18446744073709551615'
          'max_length_for_sort_data','1024'
          'max_long_data_size','33554432'
          'max_prepared_stmt_count','16382'
          'max_relay_log_size','1073741824'
          'max_seeks_for_key','4294967295'
          'max_sort_length','1024'
          'max_sp_recursion_depth','0'
          'max_tmp_tables','32'
          'max_user_connections','0'
          'max_write_lock_count','4294967295'
          'metadata_locks_cache_size','1024'
          'metadata_locks_hash_instances','8'
          'min_examined_row_limit','0'
          'mrr_buffer_size','262144'
          'multi_range_count','256'
          'myisam_block_size','1024'
          'myisam_data_pointer_size','6'
          'myisam_max_sort_file_size','9223372036853727232'
          'myisam_mmap_size','18446744073709551615'
          'myisam_recover_options','DEFAULT'
          'myisam_repair_threads','1'
          'myisam_sort_buffer_size','134216704'
          'myisam_stats_method','nulls_unequal'
          'myisam_use_mmap','OFF'
          'net_buffer_length','16384'
          'net_read_timeout','30'
          'net_retry_count','10'
          'net_write_timeout','60'
          'old','OFF'
          'old_alter_table','OFF'
          'old_mode',''
          'old_passwords','OFF'
          'open_files_limit','500005'
          'optimizer_prune_level','1'
          'optimizer_search_depth','62'
          'optimizer_selectivity_sampling_limit','100'
          'optimizer_switch','index_merge=on,index_merge_union=on,index_merge_sort_union=on,index_merge_intersection=on,index_merge_sort_intersection=off,engine_condition_pushdown=off,index_condition_pushdown=on,derived_merge=on,derived_with_keys=on,firstmatch=on,loosescan=on,materialization=on,in_to_exists=on,semijoin=on,partial_match_rowid_merge=on,partial_match_table_scan=on,subquery_cache=on,mrr=off,mrr_cost_based=off,mrr_sort_keys=off,outer_join_with_cache=on,semijoin_with_cache=on,join_cache_incremental=on,join_cache_hashed=on,join_cache_bka=on,optimize_join_buffer_size=off,table_elimination=on,extended_keys=on,exists_to_in=on'
          'optimizer_use_condition_selectivity','1'
          'performance_schema','ON'
          'performance_schema_accounts_size','100'
          'performance_schema_digests_size','10000'
          'performance_schema_events_stages_history_long_size','10000'
          'performance_schema_events_stages_history_size','10'
          'performance_schema_events_statements_history_long_size','10000'
          'performance_schema_events_statements_history_size','10'
          'performance_schema_events_waits_history_long_size','10000'
          'performance_schema_events_waits_history_size','10'
          'performance_schema_hosts_size','100'
          'performance_schema_max_cond_classes','80'
          'performance_schema_max_cond_instances','400900'
          'performance_schema_max_file_classes','50'
          'performance_schema_max_file_handles','32768'
          'performance_schema_max_file_instances','2154'
          'performance_schema_max_mutex_classes','200'
          'performance_schema_max_mutex_instances','605000'
          'performance_schema_max_rwlock_classes','40'
          'performance_schema_max_rwlock_instances','202800'
          'performance_schema_max_socket_classes','10'
          'performance_schema_max_socket_instances','200020'
          'performance_schema_max_stage_classes','150'
          'performance_schema_max_statement_classes','180'
          'performance_schema_max_table_handles','800'
          'performance_schema_max_table_instances','12500'
          'performance_schema_max_thread_classes','50'
          'performance_schema_max_thread_instances','200100'
          'performance_schema_session_connect_attrs_size','512'
          'performance_schema_setup_actors_size','100'
          'performance_schema_setup_objects_size','100'
          'performance_schema_users_size','100'
          'pid_file','/var/lib/mysql/optosql5.pid'
          'plugin_dir','/usr/lib64/mysql/plugin/'
          'plugin_maturity','unknown'
          'port','3306'
          'preload_buffer_size','32768'
          'profiling','OFF'
          'profiling_history_size','15'
          'progress_report_time','5'
          'protocol_version','10'
          'proxy_user',''
          'pseudo_slave_mode','OFF'
          'pseudo_thread_id','189143'
          'query_alloc_block_size','8192'
          'query_cache_limit','1048576'
          'query_cache_min_res_unit','4096'
          'query_cache_size','67108864'
          'query_cache_strip_comments','OFF'
          'query_cache_type','ON'
          'query_cache_wlock_invalidate','OFF'
          'query_prealloc_size','8192'
          'rand_seed1','0'
          'rand_seed2','0'
          'range_alloc_block_size','4096'
          'read_buffer_size','131072'
          'read_only','ON'
          'read_rnd_buffer_size','262144'
          'relay_log',''
          'relay_log_index',''
          'relay_log_info_file','relay-log.info'
          'relay_log_purge','ON'
          'relay_log_recovery','OFF'
          'relay_log_space_limit','0'
          'replicate_annotate_row_events','OFF'
          'replicate_do_db',''
          'replicate_do_table',''
          'replicate_events_marked_for_skip','replicate'
          'replicate_ignore_db',''
          'replicate_ignore_table',''
          'replicate_wild_do_table',''
          'replicate_wild_ignore_table',''
          'report_host',''
          'report_password',''
          'report_port','3306'
          'report_user',''
          'rowid_merge_buff_size','8388608'
          'rpl_recovery_rank','0'
          'secure_auth','OFF'
          'secure_file_priv',''
          'server_id','5'
          'skip_external_locking','ON'
          'skip_name_resolve','ON'
          'skip_networking','OFF'
          'skip_replication','OFF'
          'skip_show_database','OFF'
          'slave_compressed_protocol','OFF'
          'slave_ddl_exec_mode','IDEMPOTENT'
          'slave_domain_parallel_threads','4'
          'slave_exec_mode','STRICT'
          'slave_load_tmpdir','/tmp'
          'slave_max_allowed_packet','1073741824'
          'slave_net_timeout','3600'
          'slave_parallel_max_queued','131072'
          'slave_parallel_threads','12'
          'slave_skip_errors','OFF'
          'slave_sql_verify_checksum','ON'
          'slave_transaction_retries','10'
          'slave_type_conversions',''
          'slow_launch_time','2'
          'slow_query_log','ON'
          'slow_query_log_file','optosql5-slow.log'
          'socket','/var/lib/mysql/mysql.sock'
          'sort_buffer_size','2097152'
          'sql_auto_is_null','OFF'
          'sql_big_selects','ON'
          'sql_buffer_result','OFF'
          'sql_log_bin','ON'
          'sql_log_off','OFF'
          'sql_mode',''
          'sql_notes','ON'
          'sql_quote_show_create','ON'
          'sql_safe_updates','OFF'
          'sql_select_limit','18446744073709551615'
          'sql_slave_skip_counter','0'
          'sql_warnings','OFF'
          'ssl_ca',''
          'ssl_capath',''
          'ssl_cert',''
          'ssl_cipher',''
          'ssl_crl',''
          'ssl_crlpath',''
          'ssl_key',''
          'storage_engine','InnoDB'
          'stored_program_cache','256'
          'sync_binlog','0'
          'sync_frm','ON'
          'sync_master_info','0'
          'sync_relay_log','0'
          'sync_relay_log_info','0'
          'system_time_zone','CEST'
          'table_definition_cache','400'
          'table_open_cache','512'
          'thread_cache_size','4'
          'thread_concurrency','10'
          'thread_handling','one-thread-per-connection'
          'thread_pool_idle_timeout','60'
          'thread_pool_max_threads','500'
          'thread_pool_oversubscribe','3'
          'thread_pool_size','16'
          'thread_pool_stall_limit','500'
          'thread_stack','294912'
          'time_format','%H:%i:%s'
          'time_zone','SYSTEM'
          'timed_mutexes','OFF'
          'timestamp','1434011008.405971'
          'tmp_table_size','134217728'
          'tmpdir','/tmp'
          'transaction_alloc_block_size','8192'
          'transaction_prealloc_size','4096'
          'tx_isolation','REPEATABLE-READ'
          'tx_read_only','OFF'
          'unique_checks','ON'
          'updatable_views_with_limit','YES'
          'use_stat_tables','NEVER'
          'userstat','OFF'
          'version','10.0.16-MariaDB-log'
          'version_comment','MariaDB Server'
          'version_compile_machine','x86_64'
          'version_compile_os','Linux'
          'version_malloc_library','bundled jemalloc'
          'wait_timeout','28800'
          'warning_count','0'
          }}

          The error occurs about once a week during normal operation.

          If you need further information, or if you want us to test something, please let me know.

          Best regards,

          Thomas Mischke
          thomas.mischke Thomas Mischke made changes -
          Description Hello,

          we are having a random "duplicate key" problem with parallel replication. We were trying hard to reproduce it on our test servers as well, but currently it only happens on our live servers.

          The error we get is the following:

          150604 2:56:41 [ERROR] Slave SQL: Error 'Duplicate entry 'VerifyOrder_0-249' for key 'identry_idqueue'' on query. Default database: 'optovision'. Query: 'insert into `queueentry`(`identry`,`idqueue`,`entrydate`,`removedate`,`info`,`prio`,`reserved`,`ObjectContext`,`OriginHost`,`ReservingHost`,`UnreserveDate`,`ReservationCounter`,`OptimisticLockField`)values('VerifyOrder_0',249,'2015-06-04 02:56:40.687','2015-06-04 03:11:40.671','',0,null,'semaphore','172.16.185.22',null,null,0,0)', Gtid 0-3-454870355, Internal MariaDB error code: 1062
          150604 2:56:41 [Warning] Slave: Duplicate entry 'VerifyOrder_0-249' for key 'identry_idqueue' Error_code: 1062
          150604 2:56:41 [ERROR] Error running query, slave SQL thread aborted. Fix the problem, and restart the slave SQL thread with "SLAVE START". We stopped at log 'mysql-bin.007762' position 645226271; GTID position '77-3-552,0-3-454870353,3-3-2,2-3-469046,17-3-180,1-3-7'
          150604 2:56:41 [Note] Error reading relay log event: slave SQL thread was killed

          It is always the same table that is involved in this issue, and in all cases we analyzed in deep, it was the same value for the unique key to be written.

          The replication is based on one master and one slave, no restrictions on the data to be transferred, and no direct write requests to the slave. Once a day the slave is stopped for a short moment to synchronize the file system and take a backup on LVM level. At the moment we see no connection between this backup and the error (the point of time it happens seems to be independent)

          The table that is involved is defined as follows:

          {{
          CREATE TABLE `queueentry` (
          `OID` INT(11) NOT NULL AUTO_INCREMENT,
          `identry` VARCHAR(100) NOT NULL,
          `entrydate` DATETIME NULL DEFAULT NULL,
          `removedate` DATETIME NULL DEFAULT NULL,
          `info` VARCHAR(8096) NULL DEFAULT NULL,
          `prio` INT(11) NULL DEFAULT NULL,
          `reserved` DATETIME NULL DEFAULT NULL,
          `idqueue` INT(11) NOT NULL,
          `ObjectContext` VARCHAR(200) NULL DEFAULT NULL,
          `OriginHost` VARCHAR(200) NULL DEFAULT NULL,
          `ReservingHost` VARCHAR(200) NULL DEFAULT NULL,
          `UnreserveDate` DATETIME NULL DEFAULT NULL,
          `ReservationCounter` INT(10) UNSIGNED NOT NULL DEFAULT '0',
          `OptimisticLockField` INT(11) NULL DEFAULT NULL,
          PRIMARY KEY (`OID`),
          UNIQUE INDEX `identry_idqueue` (`identry`, `idqueue`),
          INDEX `idqueue` (`idqueue`),
          INDEX `entrydate_prio_reserved_idqueue` (`idqueue`, `reserved`, `prio`, `entrydate`),
          INDEX `removedate` (`removedate`),
          INDEX `entrydate` (`entrydate`),
          INDEX `UnreserveDate_ReservationCounter` (`ReservationCounter`, `UnreserveDate`)
          )
          COLLATE='utf8_general_ci'
          ENGINE=InnoDB
          AUTO_INCREMENT=8164332;
          }}

          We analyzed the binlog of the master in those cases and found the following:
          * Just before this insert statement there are two transactions bound together for parallel replication.
          * Both of these transactions work on the same table as the insert does.
          * The second of these transactions deletes the entry with the unique key, that will be written by the statement showing the duplicate key error.

          That is the following two statements are bound together:
          {{
          insert into `queueentry`(`identry`,`idqueue`,`entrydate`,`removedate`,`info`,`prio`,`reserved`,`ObjectContext`,`OriginHost`,`ReservingHost`,`UnreserveDate`,`ReservationCounter`,`OptimisticLockField`)values('SME_PrintLabel_0',249,'2015-06-04 02:56:40.671','2015-06-04 03:11:40.656','',0,null,'semaphore','172.16.185.21',null,null,0,0)
          delete from `queueentry` where ((`OID` = 6363790) and (`OptimisticLockField` = 0))
          }}

          And then this next transaction crashes:
          {{
          insert into `queueentry`(`identry`,`idqueue`,`entrydate`,`removedate`,`info`,`prio`,`reserved`,`ObjectContext`,`OriginHost`,`ReservingHost`,`UnreserveDate`,`ReservationCounter`,`OptimisticLockField`)values('VerifyOrder_0',249,'2015-06-04 02:56:40.687','2015-06-04 03:11:40.671','',0,null,'semaphore','172.16.185.22',null,null,0,0)
          }}

          Here is the (I hope) most important part of the binlog of the master:

          {{
          #150604 2:56:40 server id 3 end_log_pos 645225814 GTID 0-3-454870353 cid=5620557227
          /*!100001 SET @@session.gtid_seq_no=454870353*//*!*/;
          BEGIN
          /*!*/;
          # at 645225814
          #150604 2:56:40 server id 3 end_log_pos 645225842 Intvar
          SET INSERT_ID=6363805/*!*/;
          # at 645225842
          #150604 2:56:40 server id 3 end_log_pos 645226244 Query thread_id=19732488 exec_time=0 error_code=0
          SET TIMESTAMP=1433379400/*!*/;
          insert into `queueentry`(`identry`,`idqueue`,`entrydate`,`removedate`,`info`,`prio`,`reserved`,`ObjectContext`,`OriginHost`,`ReservingHost`,`UnreserveDate`,`ReservationCounter`,`OptimisticLockField`)values('SME_PrintLabel_0',249,'2015-06-04 02:56:40.671','2015-06-04 03:11:40.656','',0,null,'semaphore','172.16.185.21',null,null,0,0)
          /*!*/;
          # at 645226244
          #150604 2:56:40 server id 3 end_log_pos 645226271 Xid = 5620557224
          COMMIT/*!*/;
          # at 645226271
          #150604 2:56:40 server id 3 end_log_pos 645226311 GTID 0-3-454870354 cid=5620557227
          /*!100001 SET @@session.gtid_seq_no=454870354*//*!*/;
          BEGIN
          /*!*/;
          # at 645226311
          #150604 2:56:40 server id 3 end_log_pos 645226462 Query thread_id=19732489 exec_time=0 error_code=0
          SET TIMESTAMP=1433379400/*!*/;
          delete from `queueentry` where ((`OID` = 6363790) and (`OptimisticLockField` = 0))
          /*!*/;
          # at 645226462
          #150604 2:56:40 server id 3 end_log_pos 645226489 Xid = 5620557228
          COMMIT/*!*/;
          # at 645226489
          #150604 2:56:40 server id 3 end_log_pos 645226527 GTID 0-3-454870355
          /*!100001 SET @@session.gtid_seq_no=454870355*//*!*/;
          BEGIN
          /*!*/;
          # at 645226527
          #150604 2:56:40 server id 3 end_log_pos 645226555 Intvar
          SET INSERT_ID=6363806/*!*/;
          # at 645226555
          #150604 2:56:40 server id 3 end_log_pos 645226954 Query thread_id=19732488 exec_time=0 error_code=0
          SET TIMESTAMP=1433379400/*!*/;
          insert into `queueentry`(`identry`,`idqueue`,`entrydate`,`removedate`,`info`,`prio`,`reserved`,`ObjectContext`,`OriginHost`,`ReservingHost`,`UnreserveDate`,`ReservationCounter`,`OptimisticLockField`)values('VerifyOrder_0',249,'2015-06-04 02:56:40.687','2015-06-04 03:11:40.671','',0,null,'semaphore','172.16.185.22',null,null,0,0)
          /*!*/;
          # at 645226954
          #150604 2:56:40 server id 3 end_log_pos 645226981 Xid = 5620557238
          COMMIT/*!*/;
          # at 645226981
          }}

          The following additional observations we made:
          * The replication error only happens when the slave is at least one second behind the master. So it seems to us that there is somewhere a race condition at work which only happens when the slave is working very fast on the transactions from the master.
          * When this error happens, the command "stop slave" does not work any longer. We have to kill the database process, since some slave transactions are still waiting on each other.
          * After a restart of the database server we examine the table, and we cannot see the malicious key in the table any longer (even before the restart it is not there). We can restart the replication without further actions.
          * When we restart the replication, the slave is normally a few minutes behind the master. While catching up, we normally get the same error again.
          * As far as we can see, it is always the value 'VerifyOrder_0' that is causing the error, even though other values are written to that table as often as that value. We find this interesting, since it is - in alphabetical order - the largest value that is being stored in that table at the moment. I have only a rough understanding about indexes in databases, but I wonder if this could be related to an index reorganization due to the previous insert and delete statements.
          * The error vanishes if we set binlog_commit_wait_count=0 on the master (which is our current workaround).

          The variables on our slave are set as follows:

          {{
          'Variable_name','Value'
          'aria_block_size','8192'
          'aria_checkpoint_interval','30'
          'aria_checkpoint_log_activity','1048576'
          'aria_force_start_after_recovery_failures','0'
          'aria_group_commit','none'
          'aria_group_commit_interval','0'
          'aria_log_file_size','1073741824'
          'aria_log_purge_type','immediate'
          'aria_max_sort_file_size','9223372036853727232'
          'aria_page_checksum','ON'
          'aria_pagecache_age_threshold','300'
          'aria_pagecache_buffer_size','134217728'
          'aria_pagecache_division_limit','100'
          'aria_pagecache_file_hash_size','512'
          'aria_recover','NORMAL'
          'aria_repair_threads','1'
          'aria_sort_buffer_size','268434432'
          'aria_stats_method','nulls_unequal'
          'aria_sync_log_dir','NEWFILE'
          'aria_used_for_temp_tables','ON'
          'auto_increment_increment','1'
          'auto_increment_offset','1'
          'autocommit','ON'
          'automatic_sp_privileges','ON'
          'back_log','150'
          'basedir','/usr'
          'big_tables','OFF'
          'binlog_annotate_row_events','OFF'
          'binlog_cache_size','32768'
          'binlog_checksum','NONE'
          'binlog_commit_wait_count','0'
          'binlog_commit_wait_usec','100000'
          'binlog_direct_non_transactional_updates','OFF'
          'binlog_format','MIXED'
          'binlog_optimize_thread_scheduling','ON'
          'binlog_stmt_cache_size','32768'
          'bulk_insert_buffer_size','8388608'
          'character_set_client','utf8'
          'character_set_connection','utf8'
          'character_set_database','utf8'
          'character_set_filesystem','binary'
          'character_set_results','utf8'
          'character_set_server','utf8'
          'character_set_system','utf8'
          'character_sets_dir','/usr/share/mysql/charsets/'
          'collation_connection','utf8_general_ci'
          'collation_database','utf8_general_ci'
          'collation_server','utf8_unicode_ci'
          'completion_type','NO_CHAIN'
          'concurrent_insert','AUTO'
          'connect_timeout','10'
          'datadir','/var/lib/mysql/'
          'date_format','%Y-%m-%d'
          'datetime_format','%Y-%m-%d %H:%i:%s'
          'deadlock_search_depth_long','15'
          'deadlock_search_depth_short','4'
          'deadlock_timeout_long','50000000'
          'deadlock_timeout_short','10000'
          'debug_no_thread_alarm','OFF'
          'default_master_connection',''
          'default_regex_flags',''
          'default_storage_engine','InnoDB'
          'default_week_format','0'
          'delay_key_write','ON'
          'delayed_insert_limit','100'
          'delayed_insert_timeout','300'
          'delayed_queue_size','1000'
          'div_precision_increment','4'
          'error_count','0'
          'event_scheduler','OFF'
          'expensive_subquery_limit','100'
          'expire_logs_days','5'
          'external_user',''
          'extra_max_connections','1'
          'extra_port','0'
          'flush','OFF'
          'flush_time','0'
          'foreign_key_checks','ON'
          'ft_boolean_syntax','+ -><()~*:""&|'
          'ft_max_word_len','84'
          'ft_min_word_len','4'
          'ft_query_expansion_limit','20'
          'ft_stopword_file','(built-in)'
          'general_log','OFF'
          'general_log_file','optosql5.log'
          'group_concat_max_len','1024'
          'gtid_binlog_pos','0-3-481964408,3-3-2,2-3-504931'
          'gtid_binlog_state','0-5-479038918,0-3-481964408,3-3-2,2-3-504931'
          'gtid_current_pos','77-3-552,0-3-481964408,3-3-2,2-3-504931,17-3-180,1-3-7'
          'gtid_domain_id','0'
          'gtid_ignore_duplicates','OFF'
          'gtid_seq_no','0'
          'gtid_slave_pos','77-3-552,0-3-481964408,3-3-2,2-3-504931,17-3-180,1-3-7'
          'gtid_strict_mode','OFF'
          'have_compress','YES'
          'have_crypt','YES'
          'have_dynamic_loading','YES'
          'have_geometry','YES'
          'have_openssl','YES'
          'have_profiling','YES'
          'have_query_cache','YES'
          'have_rtree_keys','YES'
          'have_ssl','DISABLED'
          'have_symlink','DISABLED'
          'histogram_size','0'
          'histogram_type','SINGLE_PREC_HB'
          'host_cache_size','128'
          'hostname','optosql5'
          'identity','0'
          'ignore_builtin_innodb','OFF'
          'ignore_db_dirs',''
          'in_transaction','0'
          'init_connect',''
          'init_file',''
          'init_slave',''
          'innodb_adaptive_flushing','ON'
          'innodb_adaptive_flushing_lwm','10.000000'
          'innodb_adaptive_hash_index','ON'
          'innodb_adaptive_hash_index_partitions','1'
          'innodb_adaptive_max_sleep_delay','150000'
          'innodb_additional_mem_pool_size','8388608'
          'innodb_api_bk_commit_interval','5'
          'innodb_api_disable_rowlock','OFF'
          'innodb_api_enable_binlog','OFF'
          'innodb_api_enable_mdl','OFF'
          'innodb_api_trx_level','0'
          'innodb_autoextend_increment','64'
          'innodb_autoinc_lock_mode','1'
          'innodb_buffer_pool_dump_at_shutdown','OFF'
          'innodb_buffer_pool_dump_now','OFF'
          'innodb_buffer_pool_filename','ib_buffer_pool'
          'innodb_buffer_pool_instances','8'
          'innodb_buffer_pool_load_abort','OFF'
          'innodb_buffer_pool_load_at_startup','OFF'
          'innodb_buffer_pool_load_now','OFF'
          'innodb_buffer_pool_populate','OFF'
          'innodb_buffer_pool_size','34359738368'
          'innodb_change_buffer_max_size','25'
          'innodb_change_buffering','all'
          'innodb_checksum_algorithm','innodb'
          'innodb_checksums','ON'
          'innodb_cleaner_lsn_age_factor','high_checkpoint'
          'innodb_cmp_per_index_enabled','OFF'
          'innodb_commit_concurrency','0'
          'innodb_compression_failure_threshold_pct','5'
          'innodb_compression_level','6'
          'innodb_compression_pad_pct_max','50'
          'innodb_concurrency_tickets','5000'
          'innodb_corrupt_table_action','assert'
          'innodb_data_file_path','ibdata1:12M:autoextend'
          'innodb_data_home_dir',''
          'innodb_disable_sort_file_cache','OFF'
          'innodb_doublewrite','ON'
          'innodb_empty_free_list_algorithm','backoff'
          'innodb_fake_changes','OFF'
          'innodb_fast_shutdown','1'
          'innodb_file_format','Antelope'
          'innodb_file_format_check','ON'
          'innodb_file_format_max','Antelope'
          'innodb_file_per_table','ON'
          'innodb_flush_log_at_timeout','1'
          'innodb_flush_log_at_trx_commit','1'
          'innodb_flush_method',''
          'innodb_flush_neighbors','1'
          'innodb_flushing_avg_loops','30'
          'innodb_force_load_corrupted','OFF'
          'innodb_force_recovery','0'
          'innodb_foreground_preflush','exponential_backoff'
          'innodb_ft_aux_table',''
          'innodb_ft_cache_size','8000000'
          'innodb_ft_enable_diag_print','OFF'
          'innodb_ft_enable_stopword','ON'
          'innodb_ft_max_token_size','84'
          'innodb_ft_min_token_size','3'
          'innodb_ft_num_word_optimize','2000'
          'innodb_ft_result_cache_limit','2000000000'
          'innodb_ft_server_stopword_table',''
          'innodb_ft_sort_pll_degree','2'
          'innodb_ft_total_cache_size','640000000'
          'innodb_ft_user_stopword_table',''
          'innodb_io_capacity','200'
          'innodb_io_capacity_max','2000'
          'innodb_kill_idle_transaction','0'
          'innodb_large_prefix','OFF'
          'innodb_lock_wait_timeout','50'
          'innodb_locking_fake_changes','ON'
          'innodb_locks_unsafe_for_binlog','OFF'
          'innodb_log_arch_dir','./'
          'innodb_log_arch_expire_sec','0'
          'innodb_log_archive','OFF'
          'innodb_log_block_size','512'
          'innodb_log_buffer_size','8388608'
          'innodb_log_checksum_algorithm','innodb'
          'innodb_log_compressed_pages','ON'
          'innodb_log_file_size','104857600'
          'innodb_log_files_in_group','2'
          'innodb_log_group_home_dir','./'
          'innodb_lru_scan_depth','1024'
          'innodb_max_bitmap_file_size','104857600'
          'innodb_max_changed_pages','1000000'
          'innodb_max_dirty_pages_pct','75.000000'
          'innodb_max_dirty_pages_pct_lwm','0.001000'
          'innodb_max_purge_lag','0'
          'innodb_max_purge_lag_delay','0'
          'innodb_mirrored_log_groups','1'
          'innodb_monitor_disable',''
          'innodb_monitor_enable',''
          'innodb_monitor_reset',''
          'innodb_monitor_reset_all',''
          'innodb_old_blocks_pct','37'
          'innodb_old_blocks_time','1000'
          'innodb_online_alter_log_max_size','134217728'
          'innodb_open_files','512'
          'innodb_optimize_fulltext_only','OFF'
          'innodb_page_size','16384'
          'innodb_print_all_deadlocks','OFF'
          'innodb_purge_batch_size','300'
          'innodb_purge_threads','1'
          'innodb_random_read_ahead','OFF'
          'innodb_read_ahead_threshold','56'
          'innodb_read_io_threads','64'
          'innodb_read_only','OFF'
          'innodb_replication_delay','0'
          'innodb_rollback_on_timeout','OFF'
          'innodb_rollback_segments','128'
          'innodb_sched_priority_cleaner','19'
          'innodb_show_locks_held','10'
          'innodb_show_verbose_locks','0'
          'innodb_simulate_comp_failures','0'
          'innodb_sort_buffer_size','1048576'
          'innodb_spin_wait_delay','6'
          'innodb_stats_auto_recalc','ON'
          'innodb_stats_method','nulls_equal'
          'innodb_stats_modified_counter','0'
          'innodb_stats_on_metadata','OFF'
          'innodb_stats_persistent','ON'
          'innodb_stats_persistent_sample_pages','20'
          'innodb_stats_sample_pages','8'
          'innodb_stats_traditional','ON'
          'innodb_stats_transient_sample_pages','8'
          'innodb_status_output','OFF'
          'innodb_status_output_locks','OFF'
          'innodb_strict_mode','OFF'
          'innodb_support_xa','ON'
          'innodb_sync_array_size','1'
          'innodb_sync_spin_loops','30'
          'innodb_table_locks','ON'
          'innodb_thread_concurrency','0'
          'innodb_thread_sleep_delay','10000'
          'innodb_track_changed_pages','OFF'
          'innodb_undo_directory','.'
          'innodb_undo_logs','128'
          'innodb_undo_tablespaces','0'
          'innodb_use_atomic_writes','OFF'
          'innodb_use_fallocate','OFF'
          'innodb_use_global_flush_log_at_trx_commit','ON'
          'innodb_use_native_aio','ON'
          'innodb_use_stacktrace','OFF'
          'innodb_use_sys_malloc','ON'
          'innodb_version','5.6.22-71.0'
          'innodb_write_io_threads','64'
          'insert_id','0'
          'interactive_timeout','28800'
          'join_buffer_size','131072'
          'join_buffer_space_limit','2097152'
          'join_cache_level','2'
          'keep_files_on_create','OFF'
          'key_buffer_size','134217728'
          'key_cache_age_threshold','300'
          'key_cache_block_size','1024'
          'key_cache_division_limit','100'
          'key_cache_file_hash_size','512'
          'key_cache_segments','0'
          'large_files_support','ON'
          'large_page_size','0'
          'large_pages','OFF'
          'last_gtid',''
          'last_insert_id','0'
          'lc_messages','en_US'
          'lc_messages_dir',''
          'lc_time_names','en_US'
          'license','GPL'
          'local_infile','ON'
          'lock_wait_timeout','31536000'
          'locked_in_memory','OFF'
          'log_bin','ON'
          'log_bin_trust_function_creators','OFF'
          'log_error','/var/log/mysqld.log'
          'log_output','FILE'
          'log_queries_not_using_indexes','OFF'
          'log_slave_updates','ON'
          'log_slow_filter','admin,filesort,filesort_on_disk,full_join,full_scan,query_cache,query_cache_miss,tmp_table,tmp_table_on_disk'
          'log_slow_rate_limit','1'
          'log_slow_verbosity',''
          'log_warnings','1'
          'long_query_time','10.000000'
          'low_priority_updates','OFF'
          'lower_case_file_system','OFF'
          'lower_case_table_names','1'
          'master_verify_checksum','OFF'
          'max_allowed_packet','33554432'
          'max_binlog_cache_size','18446744073709547520'
          'max_binlog_size','1073741824'
          'max_binlog_stmt_cache_size','18446744073709547520'
          'max_connect_errors','100'
          'max_connections','500'
          'max_delayed_threads','20'
          'max_error_count','64'
          'max_heap_table_size','67108864'
          'max_insert_delayed_threads','20'
          'max_join_size','18446744073709551615'
          'max_length_for_sort_data','1024'
          'max_long_data_size','33554432'
          'max_prepared_stmt_count','16382'
          'max_relay_log_size','1073741824'
          'max_seeks_for_key','4294967295'
          'max_sort_length','1024'
          'max_sp_recursion_depth','0'
          'max_tmp_tables','32'
          'max_user_connections','0'
          'max_write_lock_count','4294967295'
          'metadata_locks_cache_size','1024'
          'metadata_locks_hash_instances','8'
          'min_examined_row_limit','0'
          'mrr_buffer_size','262144'
          'multi_range_count','256'
          'myisam_block_size','1024'
          'myisam_data_pointer_size','6'
          'myisam_max_sort_file_size','9223372036853727232'
          'myisam_mmap_size','18446744073709551615'
          'myisam_recover_options','DEFAULT'
          'myisam_repair_threads','1'
          'myisam_sort_buffer_size','134216704'
          'myisam_stats_method','nulls_unequal'
          'myisam_use_mmap','OFF'
          'net_buffer_length','16384'
          'net_read_timeout','30'
          'net_retry_count','10'
          'net_write_timeout','60'
          'old','OFF'
          'old_alter_table','OFF'
          'old_mode',''
          'old_passwords','OFF'
          'open_files_limit','500005'
          'optimizer_prune_level','1'
          'optimizer_search_depth','62'
          'optimizer_selectivity_sampling_limit','100'
          'optimizer_switch','index_merge=on,index_merge_union=on,index_merge_sort_union=on,index_merge_intersection=on,index_merge_sort_intersection=off,engine_condition_pushdown=off,index_condition_pushdown=on,derived_merge=on,derived_with_keys=on,firstmatch=on,loosescan=on,materialization=on,in_to_exists=on,semijoin=on,partial_match_rowid_merge=on,partial_match_table_scan=on,subquery_cache=on,mrr=off,mrr_cost_based=off,mrr_sort_keys=off,outer_join_with_cache=on,semijoin_with_cache=on,join_cache_incremental=on,join_cache_hashed=on,join_cache_bka=on,optimize_join_buffer_size=off,table_elimination=on,extended_keys=on,exists_to_in=on'
          'optimizer_use_condition_selectivity','1'
          'performance_schema','ON'
          'performance_schema_accounts_size','100'
          'performance_schema_digests_size','10000'
          'performance_schema_events_stages_history_long_size','10000'
          'performance_schema_events_stages_history_size','10'
          'performance_schema_events_statements_history_long_size','10000'
          'performance_schema_events_statements_history_size','10'
          'performance_schema_events_waits_history_long_size','10000'
          'performance_schema_events_waits_history_size','10'
          'performance_schema_hosts_size','100'
          'performance_schema_max_cond_classes','80'
          'performance_schema_max_cond_instances','400900'
          'performance_schema_max_file_classes','50'
          'performance_schema_max_file_handles','32768'
          'performance_schema_max_file_instances','2154'
          'performance_schema_max_mutex_classes','200'
          'performance_schema_max_mutex_instances','605000'
          'performance_schema_max_rwlock_classes','40'
          'performance_schema_max_rwlock_instances','202800'
          'performance_schema_max_socket_classes','10'
          'performance_schema_max_socket_instances','200020'
          'performance_schema_max_stage_classes','150'
          'performance_schema_max_statement_classes','180'
          'performance_schema_max_table_handles','800'
          'performance_schema_max_table_instances','12500'
          'performance_schema_max_thread_classes','50'
          'performance_schema_max_thread_instances','200100'
          'performance_schema_session_connect_attrs_size','512'
          'performance_schema_setup_actors_size','100'
          'performance_schema_setup_objects_size','100'
          'performance_schema_users_size','100'
          'pid_file','/var/lib/mysql/optosql5.pid'
          'plugin_dir','/usr/lib64/mysql/plugin/'
          'plugin_maturity','unknown'
          'port','3306'
          'preload_buffer_size','32768'
          'profiling','OFF'
          'profiling_history_size','15'
          'progress_report_time','5'
          'protocol_version','10'
          'proxy_user',''
          'pseudo_slave_mode','OFF'
          'pseudo_thread_id','189143'
          'query_alloc_block_size','8192'
          'query_cache_limit','1048576'
          'query_cache_min_res_unit','4096'
          'query_cache_size','67108864'
          'query_cache_strip_comments','OFF'
          'query_cache_type','ON'
          'query_cache_wlock_invalidate','OFF'
          'query_prealloc_size','8192'
          'rand_seed1','0'
          'rand_seed2','0'
          'range_alloc_block_size','4096'
          'read_buffer_size','131072'
          'read_only','ON'
          'read_rnd_buffer_size','262144'
          'relay_log',''
          'relay_log_index',''
          'relay_log_info_file','relay-log.info'
          'relay_log_purge','ON'
          'relay_log_recovery','OFF'
          'relay_log_space_limit','0'
          'replicate_annotate_row_events','OFF'
          'replicate_do_db',''
          'replicate_do_table',''
          'replicate_events_marked_for_skip','replicate'
          'replicate_ignore_db',''
          'replicate_ignore_table',''
          'replicate_wild_do_table',''
          'replicate_wild_ignore_table',''
          'report_host',''
          'report_password',''
          'report_port','3306'
          'report_user',''
          'rowid_merge_buff_size','8388608'
          'rpl_recovery_rank','0'
          'secure_auth','OFF'
          'secure_file_priv',''
          'server_id','5'
          'skip_external_locking','ON'
          'skip_name_resolve','ON'
          'skip_networking','OFF'
          'skip_replication','OFF'
          'skip_show_database','OFF'
          'slave_compressed_protocol','OFF'
          'slave_ddl_exec_mode','IDEMPOTENT'
          'slave_domain_parallel_threads','4'
          'slave_exec_mode','STRICT'
          'slave_load_tmpdir','/tmp'
          'slave_max_allowed_packet','1073741824'
          'slave_net_timeout','3600'
          'slave_parallel_max_queued','131072'
          'slave_parallel_threads','12'
          'slave_skip_errors','OFF'
          'slave_sql_verify_checksum','ON'
          'slave_transaction_retries','10'
          'slave_type_conversions',''
          'slow_launch_time','2'
          'slow_query_log','ON'
          'slow_query_log_file','optosql5-slow.log'
          'socket','/var/lib/mysql/mysql.sock'
          'sort_buffer_size','2097152'
          'sql_auto_is_null','OFF'
          'sql_big_selects','ON'
          'sql_buffer_result','OFF'
          'sql_log_bin','ON'
          'sql_log_off','OFF'
          'sql_mode',''
          'sql_notes','ON'
          'sql_quote_show_create','ON'
          'sql_safe_updates','OFF'
          'sql_select_limit','18446744073709551615'
          'sql_slave_skip_counter','0'
          'sql_warnings','OFF'
          'ssl_ca',''
          'ssl_capath',''
          'ssl_cert',''
          'ssl_cipher',''
          'ssl_crl',''
          'ssl_crlpath',''
          'ssl_key',''
          'storage_engine','InnoDB'
          'stored_program_cache','256'
          'sync_binlog','0'
          'sync_frm','ON'
          'sync_master_info','0'
          'sync_relay_log','0'
          'sync_relay_log_info','0'
          'system_time_zone','CEST'
          'table_definition_cache','400'
          'table_open_cache','512'
          'thread_cache_size','4'
          'thread_concurrency','10'
          'thread_handling','one-thread-per-connection'
          'thread_pool_idle_timeout','60'
          'thread_pool_max_threads','500'
          'thread_pool_oversubscribe','3'
          'thread_pool_size','16'
          'thread_pool_stall_limit','500'
          'thread_stack','294912'
          'time_format','%H:%i:%s'
          'time_zone','SYSTEM'
          'timed_mutexes','OFF'
          'timestamp','1434011008.405971'
          'tmp_table_size','134217728'
          'tmpdir','/tmp'
          'transaction_alloc_block_size','8192'
          'transaction_prealloc_size','4096'
          'tx_isolation','REPEATABLE-READ'
          'tx_read_only','OFF'
          'unique_checks','ON'
          'updatable_views_with_limit','YES'
          'use_stat_tables','NEVER'
          'userstat','OFF'
          'version','10.0.16-MariaDB-log'
          'version_comment','MariaDB Server'
          'version_compile_machine','x86_64'
          'version_compile_os','Linux'
          'version_malloc_library','bundled jemalloc'
          'wait_timeout','28800'
          'warning_count','0'
          }}

          The error occurs about once a week during normal operation.

          If you need further information, or if you want us to test something, please let me know.

          Best regards,

          Thomas Mischke
          Hello,

          we are having a random "duplicate key" problem with parallel replication. We were trying hard to reproduce it on our test servers as well, but currently it only happens on our live servers.

          The error we get is the following:

          {noformat}
          150604 2:56:41 [ERROR] Slave SQL: Error 'Duplicate entry 'VerifyOrder_0-249' for key 'identry_idqueue'' on query. Default database: 'optovision'. Query: 'insert into `queueentry`(`identry`,`idqueue`,`entrydate`,`removedate`,`info`,`prio`,`reserved`,`ObjectContext`,`OriginHost`,`ReservingHost`,`UnreserveDate`,`ReservationCounter`,`OptimisticLockField`)values('VerifyOrder_0',249,'2015-06-04 02:56:40.687','2015-06-04 03:11:40.671','',0,null,'semaphore','172.16.185.22',null,null,0,0)', Gtid 0-3-454870355, Internal MariaDB error code: 1062
          150604 2:56:41 [Warning] Slave: Duplicate entry 'VerifyOrder_0-249' for key 'identry_idqueue' Error_code: 1062
          150604 2:56:41 [ERROR] Error running query, slave SQL thread aborted. Fix the problem, and restart the slave SQL thread with "SLAVE START". We stopped at log 'mysql-bin.007762' position 645226271; GTID position '77-3-552,0-3-454870353,3-3-2,2-3-469046,17-3-180,1-3-7'
          150604 2:56:41 [Note] Error reading relay log event: slave SQL thread was killed
          {noformat}

          It is always the same table that is involved in this issue, and in all cases we analyzed in deep, it was the same value for the unique key to be written.

          The replication is based on one master and one slave, no restrictions on the data to be transferred, and no direct write requests to the slave. Once a day the slave is stopped for a short moment to synchronize the file system and take a backup on LVM level. At the moment we see no connection between this backup and the error (the point of time it happens seems to be independent)

          The table that is involved is defined as follows:

          {noformat}
          CREATE TABLE `queueentry` (
          `OID` INT(11) NOT NULL AUTO_INCREMENT,
          `identry` VARCHAR(100) NOT NULL,
          `entrydate` DATETIME NULL DEFAULT NULL,
          `removedate` DATETIME NULL DEFAULT NULL,
          `info` VARCHAR(8096) NULL DEFAULT NULL,
          `prio` INT(11) NULL DEFAULT NULL,
          `reserved` DATETIME NULL DEFAULT NULL,
          `idqueue` INT(11) NOT NULL,
          `ObjectContext` VARCHAR(200) NULL DEFAULT NULL,
          `OriginHost` VARCHAR(200) NULL DEFAULT NULL,
          `ReservingHost` VARCHAR(200) NULL DEFAULT NULL,
          `UnreserveDate` DATETIME NULL DEFAULT NULL,
          `ReservationCounter` INT(10) UNSIGNED NOT NULL DEFAULT '0',
          `OptimisticLockField` INT(11) NULL DEFAULT NULL,
          PRIMARY KEY (`OID`),
          UNIQUE INDEX `identry_idqueue` (`identry`, `idqueue`),
          INDEX `idqueue` (`idqueue`),
          INDEX `entrydate_prio_reserved_idqueue` (`idqueue`, `reserved`, `prio`, `entrydate`),
          INDEX `removedate` (`removedate`),
          INDEX `entrydate` (`entrydate`),
          INDEX `UnreserveDate_ReservationCounter` (`ReservationCounter`, `UnreserveDate`)
          )
          COLLATE='utf8_general_ci'
          ENGINE=InnoDB
          AUTO_INCREMENT=8164332;
          {noformat}

          We analyzed the binlog of the master in those cases and found the following:
          * Just before this insert statement there are two transactions bound together for parallel replication.
          * Both of these transactions work on the same table as the insert does.
          * The second of these transactions deletes the entry with the unique key, that will be written by the statement showing the duplicate key error.

          That is the following two statements are bound together:
          {noformat}
          insert into `queueentry`(`identry`,`idqueue`,`entrydate`,`removedate`,`info`,`prio`,`reserved`,`ObjectContext`,`OriginHost`,`ReservingHost`,`UnreserveDate`,`ReservationCounter`,`OptimisticLockField`)values('SME_PrintLabel_0',249,'2015-06-04 02:56:40.671','2015-06-04 03:11:40.656','',0,null,'semaphore','172.16.185.21',null,null,0,0)
          delete from `queueentry` where ((`OID` = 6363790) and (`OptimisticLockField` = 0))
          {noformat}

          And then this next transaction crashes:
          {noformat}
          insert into `queueentry`(`identry`,`idqueue`,`entrydate`,`removedate`,`info`,`prio`,`reserved`,`ObjectContext`,`OriginHost`,`ReservingHost`,`UnreserveDate`,`ReservationCounter`,`OptimisticLockField`)values('VerifyOrder_0',249,'2015-06-04 02:56:40.687','2015-06-04 03:11:40.671','',0,null,'semaphore','172.16.185.22',null,null,0,0)
          {noformat}

          Here is the (I hope) most important part of the binlog of the master:

          {noformat}
          #150604 2:56:40 server id 3 end_log_pos 645225814 GTID 0-3-454870353 cid=5620557227
          /*!100001 SET @@session.gtid_seq_no=454870353*//*!*/;
          BEGIN
          /*!*/;
          # at 645225814
          #150604 2:56:40 server id 3 end_log_pos 645225842 Intvar
          SET INSERT_ID=6363805/*!*/;
          # at 645225842
          #150604 2:56:40 server id 3 end_log_pos 645226244 Query thread_id=19732488 exec_time=0 error_code=0
          SET TIMESTAMP=1433379400/*!*/;
          insert into `queueentry`(`identry`,`idqueue`,`entrydate`,`removedate`,`info`,`prio`,`reserved`,`ObjectContext`,`OriginHost`,`ReservingHost`,`UnreserveDate`,`ReservationCounter`,`OptimisticLockField`)values('SME_PrintLabel_0',249,'2015-06-04 02:56:40.671','2015-06-04 03:11:40.656','',0,null,'semaphore','172.16.185.21',null,null,0,0)
          /*!*/;
          # at 645226244
          #150604 2:56:40 server id 3 end_log_pos 645226271 Xid = 5620557224
          COMMIT/*!*/;
          # at 645226271
          #150604 2:56:40 server id 3 end_log_pos 645226311 GTID 0-3-454870354 cid=5620557227
          /*!100001 SET @@session.gtid_seq_no=454870354*//*!*/;
          BEGIN
          /*!*/;
          # at 645226311
          #150604 2:56:40 server id 3 end_log_pos 645226462 Query thread_id=19732489 exec_time=0 error_code=0
          SET TIMESTAMP=1433379400/*!*/;
          delete from `queueentry` where ((`OID` = 6363790) and (`OptimisticLockField` = 0))
          /*!*/;
          # at 645226462
          #150604 2:56:40 server id 3 end_log_pos 645226489 Xid = 5620557228
          COMMIT/*!*/;
          # at 645226489
          #150604 2:56:40 server id 3 end_log_pos 645226527 GTID 0-3-454870355
          /*!100001 SET @@session.gtid_seq_no=454870355*//*!*/;
          BEGIN
          /*!*/;
          # at 645226527
          #150604 2:56:40 server id 3 end_log_pos 645226555 Intvar
          SET INSERT_ID=6363806/*!*/;
          # at 645226555
          #150604 2:56:40 server id 3 end_log_pos 645226954 Query thread_id=19732488 exec_time=0 error_code=0
          SET TIMESTAMP=1433379400/*!*/;
          insert into `queueentry`(`identry`,`idqueue`,`entrydate`,`removedate`,`info`,`prio`,`reserved`,`ObjectContext`,`OriginHost`,`ReservingHost`,`UnreserveDate`,`ReservationCounter`,`OptimisticLockField`)values('VerifyOrder_0',249,'2015-06-04 02:56:40.687','2015-06-04 03:11:40.671','',0,null,'semaphore','172.16.185.22',null,null,0,0)
          /*!*/;
          # at 645226954
          #150604 2:56:40 server id 3 end_log_pos 645226981 Xid = 5620557238
          COMMIT/*!*/;
          # at 645226981
          {noformat}

          The following additional observations we made:
          * The replication error only happens when the slave is at least one second behind the master. So it seems to us that there is somewhere a race condition at work which only happens when the slave is working very fast on the transactions from the master.
          * When this error happens, the command "stop slave" does not work any longer. We have to kill the database process, since some slave transactions are still waiting on each other.
          * After a restart of the database server we examine the table, and we cannot see the malicious key in the table any longer (even before the restart it is not there). We can restart the replication without further actions.
          * When we restart the replication, the slave is normally a few minutes behind the master. While catching up, we normally get the same error again.
          * As far as we can see, it is always the value 'VerifyOrder_0' that is causing the error, even though other values are written to that table as often as that value. We find this interesting, since it is - in alphabetical order - the largest value that is being stored in that table at the moment. I have only a rough understanding about indexes in databases, but I wonder if this could be related to an index reorganization due to the previous insert and delete statements.
          * The error vanishes if we set binlog_commit_wait_count=0 on the master (which is our current workaround).

          The variables on our slave are set as follows:

          {noformat}
          'Variable_name','Value'
          'aria_block_size','8192'
          'aria_checkpoint_interval','30'
          'aria_checkpoint_log_activity','1048576'
          'aria_force_start_after_recovery_failures','0'
          'aria_group_commit','none'
          'aria_group_commit_interval','0'
          'aria_log_file_size','1073741824'
          'aria_log_purge_type','immediate'
          'aria_max_sort_file_size','9223372036853727232'
          'aria_page_checksum','ON'
          'aria_pagecache_age_threshold','300'
          'aria_pagecache_buffer_size','134217728'
          'aria_pagecache_division_limit','100'
          'aria_pagecache_file_hash_size','512'
          'aria_recover','NORMAL'
          'aria_repair_threads','1'
          'aria_sort_buffer_size','268434432'
          'aria_stats_method','nulls_unequal'
          'aria_sync_log_dir','NEWFILE'
          'aria_used_for_temp_tables','ON'
          'auto_increment_increment','1'
          'auto_increment_offset','1'
          'autocommit','ON'
          'automatic_sp_privileges','ON'
          'back_log','150'
          'basedir','/usr'
          'big_tables','OFF'
          'binlog_annotate_row_events','OFF'
          'binlog_cache_size','32768'
          'binlog_checksum','NONE'
          'binlog_commit_wait_count','0'
          'binlog_commit_wait_usec','100000'
          'binlog_direct_non_transactional_updates','OFF'
          'binlog_format','MIXED'
          'binlog_optimize_thread_scheduling','ON'
          'binlog_stmt_cache_size','32768'
          'bulk_insert_buffer_size','8388608'
          'character_set_client','utf8'
          'character_set_connection','utf8'
          'character_set_database','utf8'
          'character_set_filesystem','binary'
          'character_set_results','utf8'
          'character_set_server','utf8'
          'character_set_system','utf8'
          'character_sets_dir','/usr/share/mysql/charsets/'
          'collation_connection','utf8_general_ci'
          'collation_database','utf8_general_ci'
          'collation_server','utf8_unicode_ci'
          'completion_type','NO_CHAIN'
          'concurrent_insert','AUTO'
          'connect_timeout','10'
          'datadir','/var/lib/mysql/'
          'date_format','%Y-%m-%d'
          'datetime_format','%Y-%m-%d %H:%i:%s'
          'deadlock_search_depth_long','15'
          'deadlock_search_depth_short','4'
          'deadlock_timeout_long','50000000'
          'deadlock_timeout_short','10000'
          'debug_no_thread_alarm','OFF'
          'default_master_connection',''
          'default_regex_flags',''
          'default_storage_engine','InnoDB'
          'default_week_format','0'
          'delay_key_write','ON'
          'delayed_insert_limit','100'
          'delayed_insert_timeout','300'
          'delayed_queue_size','1000'
          'div_precision_increment','4'
          'error_count','0'
          'event_scheduler','OFF'
          'expensive_subquery_limit','100'
          'expire_logs_days','5'
          'external_user',''
          'extra_max_connections','1'
          'extra_port','0'
          'flush','OFF'
          'flush_time','0'
          'foreign_key_checks','ON'
          'ft_boolean_syntax','+ -><()~*:""&|'
          'ft_max_word_len','84'
          'ft_min_word_len','4'
          'ft_query_expansion_limit','20'
          'ft_stopword_file','(built-in)'
          'general_log','OFF'
          'general_log_file','optosql5.log'
          'group_concat_max_len','1024'
          'gtid_binlog_pos','0-3-481964408,3-3-2,2-3-504931'
          'gtid_binlog_state','0-5-479038918,0-3-481964408,3-3-2,2-3-504931'
          'gtid_current_pos','77-3-552,0-3-481964408,3-3-2,2-3-504931,17-3-180,1-3-7'
          'gtid_domain_id','0'
          'gtid_ignore_duplicates','OFF'
          'gtid_seq_no','0'
          'gtid_slave_pos','77-3-552,0-3-481964408,3-3-2,2-3-504931,17-3-180,1-3-7'
          'gtid_strict_mode','OFF'
          'have_compress','YES'
          'have_crypt','YES'
          'have_dynamic_loading','YES'
          'have_geometry','YES'
          'have_openssl','YES'
          'have_profiling','YES'
          'have_query_cache','YES'
          'have_rtree_keys','YES'
          'have_ssl','DISABLED'
          'have_symlink','DISABLED'
          'histogram_size','0'
          'histogram_type','SINGLE_PREC_HB'
          'host_cache_size','128'
          'hostname','optosql5'
          'identity','0'
          'ignore_builtin_innodb','OFF'
          'ignore_db_dirs',''
          'in_transaction','0'
          'init_connect',''
          'init_file',''
          'init_slave',''
          'innodb_adaptive_flushing','ON'
          'innodb_adaptive_flushing_lwm','10.000000'
          'innodb_adaptive_hash_index','ON'
          'innodb_adaptive_hash_index_partitions','1'
          'innodb_adaptive_max_sleep_delay','150000'
          'innodb_additional_mem_pool_size','8388608'
          'innodb_api_bk_commit_interval','5'
          'innodb_api_disable_rowlock','OFF'
          'innodb_api_enable_binlog','OFF'
          'innodb_api_enable_mdl','OFF'
          'innodb_api_trx_level','0'
          'innodb_autoextend_increment','64'
          'innodb_autoinc_lock_mode','1'
          'innodb_buffer_pool_dump_at_shutdown','OFF'
          'innodb_buffer_pool_dump_now','OFF'
          'innodb_buffer_pool_filename','ib_buffer_pool'
          'innodb_buffer_pool_instances','8'
          'innodb_buffer_pool_load_abort','OFF'
          'innodb_buffer_pool_load_at_startup','OFF'
          'innodb_buffer_pool_load_now','OFF'
          'innodb_buffer_pool_populate','OFF'
          'innodb_buffer_pool_size','34359738368'
          'innodb_change_buffer_max_size','25'
          'innodb_change_buffering','all'
          'innodb_checksum_algorithm','innodb'
          'innodb_checksums','ON'
          'innodb_cleaner_lsn_age_factor','high_checkpoint'
          'innodb_cmp_per_index_enabled','OFF'
          'innodb_commit_concurrency','0'
          'innodb_compression_failure_threshold_pct','5'
          'innodb_compression_level','6'
          'innodb_compression_pad_pct_max','50'
          'innodb_concurrency_tickets','5000'
          'innodb_corrupt_table_action','assert'
          'innodb_data_file_path','ibdata1:12M:autoextend'
          'innodb_data_home_dir',''
          'innodb_disable_sort_file_cache','OFF'
          'innodb_doublewrite','ON'
          'innodb_empty_free_list_algorithm','backoff'
          'innodb_fake_changes','OFF'
          'innodb_fast_shutdown','1'
          'innodb_file_format','Antelope'
          'innodb_file_format_check','ON'
          'innodb_file_format_max','Antelope'
          'innodb_file_per_table','ON'
          'innodb_flush_log_at_timeout','1'
          'innodb_flush_log_at_trx_commit','1'
          'innodb_flush_method',''
          'innodb_flush_neighbors','1'
          'innodb_flushing_avg_loops','30'
          'innodb_force_load_corrupted','OFF'
          'innodb_force_recovery','0'
          'innodb_foreground_preflush','exponential_backoff'
          'innodb_ft_aux_table',''
          'innodb_ft_cache_size','8000000'
          'innodb_ft_enable_diag_print','OFF'
          'innodb_ft_enable_stopword','ON'
          'innodb_ft_max_token_size','84'
          'innodb_ft_min_token_size','3'
          'innodb_ft_num_word_optimize','2000'
          'innodb_ft_result_cache_limit','2000000000'
          'innodb_ft_server_stopword_table',''
          'innodb_ft_sort_pll_degree','2'
          'innodb_ft_total_cache_size','640000000'
          'innodb_ft_user_stopword_table',''
          'innodb_io_capacity','200'
          'innodb_io_capacity_max','2000'
          'innodb_kill_idle_transaction','0'
          'innodb_large_prefix','OFF'
          'innodb_lock_wait_timeout','50'
          'innodb_locking_fake_changes','ON'
          'innodb_locks_unsafe_for_binlog','OFF'
          'innodb_log_arch_dir','./'
          'innodb_log_arch_expire_sec','0'
          'innodb_log_archive','OFF'
          'innodb_log_block_size','512'
          'innodb_log_buffer_size','8388608'
          'innodb_log_checksum_algorithm','innodb'
          'innodb_log_compressed_pages','ON'
          'innodb_log_file_size','104857600'
          'innodb_log_files_in_group','2'
          'innodb_log_group_home_dir','./'
          'innodb_lru_scan_depth','1024'
          'innodb_max_bitmap_file_size','104857600'
          'innodb_max_changed_pages','1000000'
          'innodb_max_dirty_pages_pct','75.000000'
          'innodb_max_dirty_pages_pct_lwm','0.001000'
          'innodb_max_purge_lag','0'
          'innodb_max_purge_lag_delay','0'
          'innodb_mirrored_log_groups','1'
          'innodb_monitor_disable',''
          'innodb_monitor_enable',''
          'innodb_monitor_reset',''
          'innodb_monitor_reset_all',''
          'innodb_old_blocks_pct','37'
          'innodb_old_blocks_time','1000'
          'innodb_online_alter_log_max_size','134217728'
          'innodb_open_files','512'
          'innodb_optimize_fulltext_only','OFF'
          'innodb_page_size','16384'
          'innodb_print_all_deadlocks','OFF'
          'innodb_purge_batch_size','300'
          'innodb_purge_threads','1'
          'innodb_random_read_ahead','OFF'
          'innodb_read_ahead_threshold','56'
          'innodb_read_io_threads','64'
          'innodb_read_only','OFF'
          'innodb_replication_delay','0'
          'innodb_rollback_on_timeout','OFF'
          'innodb_rollback_segments','128'
          'innodb_sched_priority_cleaner','19'
          'innodb_show_locks_held','10'
          'innodb_show_verbose_locks','0'
          'innodb_simulate_comp_failures','0'
          'innodb_sort_buffer_size','1048576'
          'innodb_spin_wait_delay','6'
          'innodb_stats_auto_recalc','ON'
          'innodb_stats_method','nulls_equal'
          'innodb_stats_modified_counter','0'
          'innodb_stats_on_metadata','OFF'
          'innodb_stats_persistent','ON'
          'innodb_stats_persistent_sample_pages','20'
          'innodb_stats_sample_pages','8'
          'innodb_stats_traditional','ON'
          'innodb_stats_transient_sample_pages','8'
          'innodb_status_output','OFF'
          'innodb_status_output_locks','OFF'
          'innodb_strict_mode','OFF'
          'innodb_support_xa','ON'
          'innodb_sync_array_size','1'
          'innodb_sync_spin_loops','30'
          'innodb_table_locks','ON'
          'innodb_thread_concurrency','0'
          'innodb_thread_sleep_delay','10000'
          'innodb_track_changed_pages','OFF'
          'innodb_undo_directory','.'
          'innodb_undo_logs','128'
          'innodb_undo_tablespaces','0'
          'innodb_use_atomic_writes','OFF'
          'innodb_use_fallocate','OFF'
          'innodb_use_global_flush_log_at_trx_commit','ON'
          'innodb_use_native_aio','ON'
          'innodb_use_stacktrace','OFF'
          'innodb_use_sys_malloc','ON'
          'innodb_version','5.6.22-71.0'
          'innodb_write_io_threads','64'
          'insert_id','0'
          'interactive_timeout','28800'
          'join_buffer_size','131072'
          'join_buffer_space_limit','2097152'
          'join_cache_level','2'
          'keep_files_on_create','OFF'
          'key_buffer_size','134217728'
          'key_cache_age_threshold','300'
          'key_cache_block_size','1024'
          'key_cache_division_limit','100'
          'key_cache_file_hash_size','512'
          'key_cache_segments','0'
          'large_files_support','ON'
          'large_page_size','0'
          'large_pages','OFF'
          'last_gtid',''
          'last_insert_id','0'
          'lc_messages','en_US'
          'lc_messages_dir',''
          'lc_time_names','en_US'
          'license','GPL'
          'local_infile','ON'
          'lock_wait_timeout','31536000'
          'locked_in_memory','OFF'
          'log_bin','ON'
          'log_bin_trust_function_creators','OFF'
          'log_error','/var/log/mysqld.log'
          'log_output','FILE'
          'log_queries_not_using_indexes','OFF'
          'log_slave_updates','ON'
          'log_slow_filter','admin,filesort,filesort_on_disk,full_join,full_scan,query_cache,query_cache_miss,tmp_table,tmp_table_on_disk'
          'log_slow_rate_limit','1'
          'log_slow_verbosity',''
          'log_warnings','1'
          'long_query_time','10.000000'
          'low_priority_updates','OFF'
          'lower_case_file_system','OFF'
          'lower_case_table_names','1'
          'master_verify_checksum','OFF'
          'max_allowed_packet','33554432'
          'max_binlog_cache_size','18446744073709547520'
          'max_binlog_size','1073741824'
          'max_binlog_stmt_cache_size','18446744073709547520'
          'max_connect_errors','100'
          'max_connections','500'
          'max_delayed_threads','20'
          'max_error_count','64'
          'max_heap_table_size','67108864'
          'max_insert_delayed_threads','20'
          'max_join_size','18446744073709551615'
          'max_length_for_sort_data','1024'
          'max_long_data_size','33554432'
          'max_prepared_stmt_count','16382'
          'max_relay_log_size','1073741824'
          'max_seeks_for_key','4294967295'
          'max_sort_length','1024'
          'max_sp_recursion_depth','0'
          'max_tmp_tables','32'
          'max_user_connections','0'
          'max_write_lock_count','4294967295'
          'metadata_locks_cache_size','1024'
          'metadata_locks_hash_instances','8'
          'min_examined_row_limit','0'
          'mrr_buffer_size','262144'
          'multi_range_count','256'
          'myisam_block_size','1024'
          'myisam_data_pointer_size','6'
          'myisam_max_sort_file_size','9223372036853727232'
          'myisam_mmap_size','18446744073709551615'
          'myisam_recover_options','DEFAULT'
          'myisam_repair_threads','1'
          'myisam_sort_buffer_size','134216704'
          'myisam_stats_method','nulls_unequal'
          'myisam_use_mmap','OFF'
          'net_buffer_length','16384'
          'net_read_timeout','30'
          'net_retry_count','10'
          'net_write_timeout','60'
          'old','OFF'
          'old_alter_table','OFF'
          'old_mode',''
          'old_passwords','OFF'
          'open_files_limit','500005'
          'optimizer_prune_level','1'
          'optimizer_search_depth','62'
          'optimizer_selectivity_sampling_limit','100'
          'optimizer_switch','index_merge=on,index_merge_union=on,index_merge_sort_union=on,index_merge_intersection=on,index_merge_sort_intersection=off,engine_condition_pushdown=off,index_condition_pushdown=on,derived_merge=on,derived_with_keys=on,firstmatch=on,loosescan=on,materialization=on,in_to_exists=on,semijoin=on,partial_match_rowid_merge=on,partial_match_table_scan=on,subquery_cache=on,mrr=off,mrr_cost_based=off,mrr_sort_keys=off,outer_join_with_cache=on,semijoin_with_cache=on,join_cache_incremental=on,join_cache_hashed=on,join_cache_bka=on,optimize_join_buffer_size=off,table_elimination=on,extended_keys=on,exists_to_in=on'
          'optimizer_use_condition_selectivity','1'
          'performance_schema','ON'
          'performance_schema_accounts_size','100'
          'performance_schema_digests_size','10000'
          'performance_schema_events_stages_history_long_size','10000'
          'performance_schema_events_stages_history_size','10'
          'performance_schema_events_statements_history_long_size','10000'
          'performance_schema_events_statements_history_size','10'
          'performance_schema_events_waits_history_long_size','10000'
          'performance_schema_events_waits_history_size','10'
          'performance_schema_hosts_size','100'
          'performance_schema_max_cond_classes','80'
          'performance_schema_max_cond_instances','400900'
          'performance_schema_max_file_classes','50'
          'performance_schema_max_file_handles','32768'
          'performance_schema_max_file_instances','2154'
          'performance_schema_max_mutex_classes','200'
          'performance_schema_max_mutex_instances','605000'
          'performance_schema_max_rwlock_classes','40'
          'performance_schema_max_rwlock_instances','202800'
          'performance_schema_max_socket_classes','10'
          'performance_schema_max_socket_instances','200020'
          'performance_schema_max_stage_classes','150'
          'performance_schema_max_statement_classes','180'
          'performance_schema_max_table_handles','800'
          'performance_schema_max_table_instances','12500'
          'performance_schema_max_thread_classes','50'
          'performance_schema_max_thread_instances','200100'
          'performance_schema_session_connect_attrs_size','512'
          'performance_schema_setup_actors_size','100'
          'performance_schema_setup_objects_size','100'
          'performance_schema_users_size','100'
          'pid_file','/var/lib/mysql/optosql5.pid'
          'plugin_dir','/usr/lib64/mysql/plugin/'
          'plugin_maturity','unknown'
          'port','3306'
          'preload_buffer_size','32768'
          'profiling','OFF'
          'profiling_history_size','15'
          'progress_report_time','5'
          'protocol_version','10'
          'proxy_user',''
          'pseudo_slave_mode','OFF'
          'pseudo_thread_id','189143'
          'query_alloc_block_size','8192'
          'query_cache_limit','1048576'
          'query_cache_min_res_unit','4096'
          'query_cache_size','67108864'
          'query_cache_strip_comments','OFF'
          'query_cache_type','ON'
          'query_cache_wlock_invalidate','OFF'
          'query_prealloc_size','8192'
          'rand_seed1','0'
          'rand_seed2','0'
          'range_alloc_block_size','4096'
          'read_buffer_size','131072'
          'read_only','ON'
          'read_rnd_buffer_size','262144'
          'relay_log',''
          'relay_log_index',''
          'relay_log_info_file','relay-log.info'
          'relay_log_purge','ON'
          'relay_log_recovery','OFF'
          'relay_log_space_limit','0'
          'replicate_annotate_row_events','OFF'
          'replicate_do_db',''
          'replicate_do_table',''
          'replicate_events_marked_for_skip','replicate'
          'replicate_ignore_db',''
          'replicate_ignore_table',''
          'replicate_wild_do_table',''
          'replicate_wild_ignore_table',''
          'report_host',''
          'report_password',''
          'report_port','3306'
          'report_user',''
          'rowid_merge_buff_size','8388608'
          'rpl_recovery_rank','0'
          'secure_auth','OFF'
          'secure_file_priv',''
          'server_id','5'
          'skip_external_locking','ON'
          'skip_name_resolve','ON'
          'skip_networking','OFF'
          'skip_replication','OFF'
          'skip_show_database','OFF'
          'slave_compressed_protocol','OFF'
          'slave_ddl_exec_mode','IDEMPOTENT'
          'slave_domain_parallel_threads','4'
          'slave_exec_mode','STRICT'
          'slave_load_tmpdir','/tmp'
          'slave_max_allowed_packet','1073741824'
          'slave_net_timeout','3600'
          'slave_parallel_max_queued','131072'
          'slave_parallel_threads','12'
          'slave_skip_errors','OFF'
          'slave_sql_verify_checksum','ON'
          'slave_transaction_retries','10'
          'slave_type_conversions',''
          'slow_launch_time','2'
          'slow_query_log','ON'
          'slow_query_log_file','optosql5-slow.log'
          'socket','/var/lib/mysql/mysql.sock'
          'sort_buffer_size','2097152'
          'sql_auto_is_null','OFF'
          'sql_big_selects','ON'
          'sql_buffer_result','OFF'
          'sql_log_bin','ON'
          'sql_log_off','OFF'
          'sql_mode',''
          'sql_notes','ON'
          'sql_quote_show_create','ON'
          'sql_safe_updates','OFF'
          'sql_select_limit','18446744073709551615'
          'sql_slave_skip_counter','0'
          'sql_warnings','OFF'
          'ssl_ca',''
          'ssl_capath',''
          'ssl_cert',''
          'ssl_cipher',''
          'ssl_crl',''
          'ssl_crlpath',''
          'ssl_key',''
          'storage_engine','InnoDB'
          'stored_program_cache','256'
          'sync_binlog','0'
          'sync_frm','ON'
          'sync_master_info','0'
          'sync_relay_log','0'
          'sync_relay_log_info','0'
          'system_time_zone','CEST'
          'table_definition_cache','400'
          'table_open_cache','512'
          'thread_cache_size','4'
          'thread_concurrency','10'
          'thread_handling','one-thread-per-connection'
          'thread_pool_idle_timeout','60'
          'thread_pool_max_threads','500'
          'thread_pool_oversubscribe','3'
          'thread_pool_size','16'
          'thread_pool_stall_limit','500'
          'thread_stack','294912'
          'time_format','%H:%i:%s'
          'time_zone','SYSTEM'
          'timed_mutexes','OFF'
          'timestamp','1434011008.405971'
          'tmp_table_size','134217728'
          'tmpdir','/tmp'
          'transaction_alloc_block_size','8192'
          'transaction_prealloc_size','4096'
          'tx_isolation','REPEATABLE-READ'
          'tx_read_only','OFF'
          'unique_checks','ON'
          'updatable_views_with_limit','YES'
          'use_stat_tables','NEVER'
          'userstat','OFF'
          'version','10.0.16-MariaDB-log'
          'version_comment','MariaDB Server'
          'version_compile_machine','x86_64'
          'version_compile_os','Linux'
          'version_malloc_library','bundled jemalloc'
          'wait_timeout','28800'
          'warning_count','0'
          {noformat}

          The error occurs about once a week during normal operation.

          If you need further information, or if you want us to test something, please let me know.

          Best regards,

          Thomas Mischke

          thomas.mischke,

          Thanks for the detailed report.
          Just to clarify – you've set the affected version to 10.0.16-galera, is it really what you are using?
          I don't see in your variables any signs of Galera cluster, but it's possible that you are running normal replication on the Galera-enabled server. It is not an explanation for the error, but it might be useful to know.

          knielsen,

          Could you please take a look? it's quite a thorough description, and you've done a lot of work on parallel replication in 10.0.17/10.0.18, so maybe you will recognize something you have already fixed.

          elenst Elena Stepanova added a comment - thomas.mischke , Thanks for the detailed report. Just to clarify – you've set the affected version to 10.0.16-galera, is it really what you are using? I don't see in your variables any signs of Galera cluster, but it's possible that you are running normal replication on the Galera-enabled server. It is not an explanation for the error, but it might be useful to know. knielsen , Could you please take a look? it's quite a thorough description, and you've done a lot of work on parallel replication in 10.0.17/10.0.18, so maybe you will recognize something you have already fixed.
          elenst Elena Stepanova made changes -
          Fix Version/s 10.0 [ 16000 ]
          Assignee Kristian Nielsen [ knielsen ]

          > Could you please take a look? it's quite a thorough description, and
          > you've done a lot of work on parallel replication in 10.0.17/10.0.18, so
          > maybe you will recognize something you have already fixed.

          Yes, thanks Thomas for the detailed analysis.

          It seems likely that this is a consequence of MDEV-7458. MDEV-7458 is
          given as fixed in 10.0.17, so 10.0.16 would be affected.

          The scenario with MDEV-7458 is that T2 deadlocks with T1, T2 is rolled back,
          and the bug allows T3 to start before T2 has been retried. This matches the
          analysis here, where T3 gets a duplicate key error because T2 has not been
          run yet.

          This requires a lock conflict between T1 and T2. This also matches that T1
          and T2 in the bug observed here are always on the same table, and with
          specific unique key values. As Thomas remarked, this seems to be related to
          some index reorganisation or some such; the details are not fully understood
          but similar cases of very rare conflicts were observed in another busy
          installation.

          (The conflict between T1 and T2 is not a bug; the bug is that the conflict
          was incorrectly handled in some rare corner cases).

          The fact that the slave hangs after this also matches previously seen
          behaviour, MDEV-7888 and MDEV-7929. It is not obvious without detailed
          understanding of the code that these bugs are related. But the basic issue
          is that the MDEV-7458 bug, which causes T3 to run too early, triggers
          another bug (fixed with MDEV-7888 / MDEV-7929 patch) that causes the slave
          threads to hang.

          (By the way, it is possible to resolve the hang by killing (on the SQL
          level) the slave worker threads, but it might require multiple kills until
          everything gets unlocked).

          The previous instance of the above-mentioned bugs were very hard to
          reproduce in testing, as the races involved seem really rare, this also
          matches Thomas' observations.

          The error would indeed only be seen when the slave is catching up, as it
          requires T3 to be available while T1 and T2 are still executing.

          So it seems likely that upgrading to latest will solve this issue.

          Elena, feel free to decide how to proceed with this bug: close it, leave it
          open to give Thomas a chance to test at some point if an upgrade indeed
          fixes his issue, or whatever.

          knielsen Kristian Nielsen added a comment - > Could you please take a look? it's quite a thorough description, and > you've done a lot of work on parallel replication in 10.0.17/10.0.18, so > maybe you will recognize something you have already fixed. Yes, thanks Thomas for the detailed analysis. It seems likely that this is a consequence of MDEV-7458 . MDEV-7458 is given as fixed in 10.0.17, so 10.0.16 would be affected. The scenario with MDEV-7458 is that T2 deadlocks with T1, T2 is rolled back, and the bug allows T3 to start before T2 has been retried. This matches the analysis here, where T3 gets a duplicate key error because T2 has not been run yet. This requires a lock conflict between T1 and T2. This also matches that T1 and T2 in the bug observed here are always on the same table, and with specific unique key values. As Thomas remarked, this seems to be related to some index reorganisation or some such; the details are not fully understood but similar cases of very rare conflicts were observed in another busy installation. (The conflict between T1 and T2 is not a bug; the bug is that the conflict was incorrectly handled in some rare corner cases). The fact that the slave hangs after this also matches previously seen behaviour, MDEV-7888 and MDEV-7929 . It is not obvious without detailed understanding of the code that these bugs are related. But the basic issue is that the MDEV-7458 bug, which causes T3 to run too early, triggers another bug (fixed with MDEV-7888 / MDEV-7929 patch) that causes the slave threads to hang. (By the way, it is possible to resolve the hang by killing (on the SQL level) the slave worker threads, but it might require multiple kills until everything gets unlocked). The previous instance of the above-mentioned bugs were very hard to reproduce in testing, as the races involved seem really rare, this also matches Thomas' observations. The error would indeed only be seen when the slave is catching up, as it requires T3 to be available while T1 and T2 are still executing. So it seems likely that upgrading to latest will solve this issue. Elena, feel free to decide how to proceed with this bug: close it, leave it open to give Thomas a chance to test at some point if an upgrade indeed fixes his issue, or whatever.
          elenst Elena Stepanova made changes -
          Assignee Kristian Nielsen [ knielsen ] Elena Stepanova [ elenst ]

          Elena, feel free to decide how to proceed with this bug: close it, leave it
          open to give Thomas a chance to test at some point if an upgrade indeed
          fixes his issue, or whatever.

          Thanks, Kristian.
          Lets wait for feedback from Thomas (whether he's planning to upgrade, etc.).

          _Note: Gven the analysis above, my question about Galera binaries appears to be irrelavant. _

          elenst Elena Stepanova added a comment - Elena, feel free to decide how to proceed with this bug: close it, leave it open to give Thomas a chance to test at some point if an upgrade indeed fixes his issue, or whatever. Thanks, Kristian. Lets wait for feedback from Thomas (whether he's planning to upgrade, etc.). _Note: Gven the analysis above, my question about Galera binaries appears to be irrelavant. _
          elenst Elena Stepanova made changes -
          Labels parallelslave replication need_feedback parallelslave replication
          thomas.mischke Thomas Mischke made changes -
          Affects Version/s 10.0.16 [ 17900 ]
          Affects Version/s 10.0.16-galera [ 18101 ]

          Thanks for the answers. Yes, of course you are right, not Galera, just the normal version.

          We are going to upgrade the slave as soon as possible to the latest version and give it a try. Since the problem occurred not that often, and an update of the slave needs at least a weekend, it may take a few weeks before I come back with a positive or negative result.

          Thanks for the details, especially about which things to try to avoid restart of the complete server, and the relations with other bug reports. This gives me the chance to understand, what was going on (and gives me the feeling that the invested time was well invested).

          Just one more question: The server marked T1 and T2 to be independent, that means they can run in parallel on the slave (at least that was my understanding). Now the slave detects a deadlock between them and rolls back T2 to retry after T1 finished. Why cannot detect the master this situation and place T1 and T2 in two different groups of transactions?

          thomas.mischke Thomas Mischke added a comment - Thanks for the answers. Yes, of course you are right, not Galera, just the normal version. We are going to upgrade the slave as soon as possible to the latest version and give it a try. Since the problem occurred not that often, and an update of the slave needs at least a weekend, it may take a few weeks before I come back with a positive or negative result. Thanks for the details, especially about which things to try to avoid restart of the complete server, and the relations with other bug reports. This gives me the chance to understand, what was going on (and gives me the feeling that the invested time was well invested). Just one more question: The server marked T1 and T2 to be independent, that means they can run in parallel on the slave (at least that was my understanding). Now the slave detects a deadlock between them and rolls back T2 to retry after T1 finished. Why cannot detect the master this situation and place T1 and T2 in two different groups of transactions?

          > Just one more question: The server marked T1 and T2 to be independent,
          > that means they can run in parallel on the slave (at least that was my
          > understanding). Now the slave detects a deadlock between them and rolls
          > back T2 to retry after T1 finished. Why cannot detect the master this
          > situation and place T1 and T2 in two different groups of transactions?

          That's an interesting question. The background for this is in bugs like
          MDEV-5941 and MDEV-5914.

          The master decides if T1 and T2 can be put in the same group by checking if
          the row locks they take are conflicting. If not, they can be put in the same
          group. The assumption then is that the locks taken by the transactions on
          the slave will also be non-conflicting.

          However, it turns out that there are a number of corner cases where the
          locking is asymmetric. In one case T1 and T2 can run without conflicting
          locks, but in another case conflicts occur. This can depend for example on
          order of taking the locks, or on the server picking different execution
          plans, or other ways.

          Your case looks similar to another one I saw involving multiple indexes.
          I have not been able to reproduce the conflict manually, but I suspect that
          with some particular layout or reorganisation of the index btrees, the
          conflict can occur. It would be an interesting exercise to investigate
          InnoDB locking in sufficient detail to understand how the conflict occurs,
          but probably a tricky one, as well.

          In any case, there are a number of ways that such conflicts can occur on the
          slave, so the server code needs to handle it (and does). You can check how
          often (or how rarely, rather) it happens by monitoring the
          Slave_retried_transactions status variable.

          Because the conflicts happen so rarely, testing is very tricky and we had
          historically some bugs. Hopefully these are all fixed in the latest version.

          knielsen Kristian Nielsen added a comment - > Just one more question: The server marked T1 and T2 to be independent, > that means they can run in parallel on the slave (at least that was my > understanding). Now the slave detects a deadlock between them and rolls > back T2 to retry after T1 finished. Why cannot detect the master this > situation and place T1 and T2 in two different groups of transactions? That's an interesting question. The background for this is in bugs like MDEV-5941 and MDEV-5914 . The master decides if T1 and T2 can be put in the same group by checking if the row locks they take are conflicting. If not, they can be put in the same group. The assumption then is that the locks taken by the transactions on the slave will also be non-conflicting. However, it turns out that there are a number of corner cases where the locking is asymmetric. In one case T1 and T2 can run without conflicting locks, but in another case conflicts occur. This can depend for example on order of taking the locks, or on the server picking different execution plans, or other ways. Your case looks similar to another one I saw involving multiple indexes. I have not been able to reproduce the conflict manually, but I suspect that with some particular layout or reorganisation of the index btrees, the conflict can occur. It would be an interesting exercise to investigate InnoDB locking in sufficient detail to understand how the conflict occurs, but probably a tricky one, as well. In any case, there are a number of ways that such conflicts can occur on the slave, so the server code needs to handle it (and does). You can check how often (or how rarely, rather) it happens by monitoring the Slave_retried_transactions status variable. Because the conflicts happen so rarely, testing is very tricky and we had historically some bugs. Hopefully these are all fixed in the latest version.

          Thanks again.

          Just to complete the picture: Between the last restart of our slave (4th of June) and disabling parallel replication at the master (5th of June), our variable slave_transactions_retries reached the value of 10 (I guess it is reseted after each restart of MySQL). So roughly I guess that we have this case once in two hours. I will observe this once we upgraded to the latest version.

          thomas.mischke Thomas Mischke added a comment - Thanks again. Just to complete the picture: Between the last restart of our slave (4th of June) and disabling parallel replication at the master (5th of June), our variable slave_transactions_retries reached the value of 10 (I guess it is reseted after each restart of MySQL). So roughly I guess that we have this case once in two hours. I will observe this once we upgraded to the latest version.
          thomas.mischke Thomas Mischke added a comment - - edited

          I am coming back with bad news. We used the weekend to update our slave to MariaDB 10.0.19, and it took just a few hours for the error to reappear. Then we set up a second slave with MariaDB 10.0.20. Again after a few hours we got the same error. Well, "the same" does not match it 100%...

          Here are the details for the error with the version 10.0.19:

          • This time, the command "stop slave" worked, so at least something has changed, since in the old version, this command hung. During catching up the same error occurred another two times, each time "stop slave" worked.
          • It is no longer the alphabetically last value in the index, that is causing the problem. This time it was a value from somewhere in the middle, but "always the same value".
          • Again the slave was a few seconds behind the master, when the error happened, and no further transactions took place between those I write about.
          • As before the last transactions to complete were two transactions grouped together. The new thing: The slave itself grouped them together in its binlog this time. This was not the case with the old version. To me this indicates that none of these transactions was locked, killed and reexecuted.
          • This time the transaction to fail was itself a part of a group of transactions. All four cases that I took a look at had this pattern, that is every time it was part of a group of transactions, it was never a standalone transaction.

          Here is the binlog of the master. The two statements with cid=6680351921 were the last two to be executed on the slave. After that a new group of two transactions got to the slave. The "insert" statement was causing the crash.

          #150620  6:42:35 server id 3  end_log_pos 123699299     GTID 0-3-506902038 cid=6680351921
          /*!100001 SET @@session.gtid_seq_no=506902038*//*!*/;
          BEGIN
          /*!*/;
          # at 123699299
          #150620  6:42:35 server id 3  end_log_pos 123699327     Intvar
          SET INSERT_ID=10337814/*!*/;
          # at 123699327
          #150620  6:42:35 server id 3  end_log_pos 123699730     Query   thread_id=21184521      exec_time=0
               error_code=0
          SET TIMESTAMP=1434775355/*!*/;
          insert into `queueentry`(`identry`,`idqueue`,`entrydate`,`removedate`,`info`,`prio`,`reserved`,`Obje
          ctContext`,`OriginHost`,`ReservingHost`,`UnreserveDate`,`ReservationCounter`,`OptimisticLockField`)v
          alues('RemoveFromStock_0',249,'2015-06-20 06:42:35.725','2015-06-20 06:57:35.725','',0,null,'semapho
          re','172.16.185.21',null,null,0,0)
          /*!*/;
          # at 123699730
          #150620  6:42:35 server id 3  end_log_pos 123699757     Xid = 6680351917
          COMMIT/*!*/;
          # at 123699757
          #150620  6:42:35 server id 3  end_log_pos 123699797     GTID 0-3-506902039 cid=6680351921
          /*!100001 SET @@session.gtid_seq_no=506902039*//*!*/;
          BEGIN
          /*!*/;
          # at 123699797
          #150620  6:42:35 server id 3  end_log_pos 123699949     Query   thread_id=19732490      exec_time=0
               error_code=0
          SET TIMESTAMP=1434775355/*!*/;
          delete from `queueentry` where ((`OID` = 10337810) and (`OptimisticLockField` = 0))
          /*!*/;
          # at 123699949
          #150620  6:42:35 server id 3  end_log_pos 123699976     Xid = 6680351922
          COMMIT/*!*/;
          # at 123699976
          #150620  6:42:35 server id 3  end_log_pos 123700016     GTID 0-3-506902040 cid=6680351936
          /*!100001 SET @@session.gtid_seq_no=506902040*//*!*/;
          BEGIN
          /*!*/;
          # at 123700016
          #150620  6:42:35 server id 3  end_log_pos 123700478     Query   thread_id=19732490      exec_time=0
               error_code=0
          SET TIMESTAMP=1434775355/*!*/;
          update `queueentry` set `identry`='637714',`idqueue`=294,`entrydate`='2015-06-20 00:59:46 ',`removedate`='2015-06-21 00:59:46 ',`info`='',`prio`=0,`reserved`='2015-06-20 06:42:35.756',`ObjectContext`='job',`OriginHost`='172.16.185.22',`ReservingHost`='172.16.185.21',`UnreserveDate`=null,`ReservationCounter`=2,`OptimisticLockField`=4 where ((`OID` = 10283585) and (`OptimisticLockField` = 3))
          /*!*/;
          # at 123700478
          #150620  6:42:35 server id 3  end_log_pos 123700505     Xid = 6680351933
          COMMIT/*!*/;
          # at 123700505
          #150620  6:42:35 server id 3  end_log_pos 123700545     GTID 0-3-506902041 cid=6680351936
          /*!100001 SET @@session.gtid_seq_no=506902041*//*!*/;
          BEGIN
          /*!*/;
          # at 123700545
          #150620  6:42:35 server id 3  end_log_pos 123700573     Intvar
          SET INSERT_ID=10337815/*!*/;
          # at 123700573
          #150620  6:42:35 server id 3  end_log_pos 123700973     Query   thread_id=21184521      exec_time=0
               error_code=0
          SET TIMESTAMP=1434775355/*!*/;
          insert into `queueentry`(`identry`,`idqueue`,`entrydate`,`removedate`,`info`,`prio`,`reserved`,`ObjectContext`,`OriginHost`,`ReservingHost`,`UnreserveDate`,`ReservationCounter`,`OptimisticLockField`)values('ReserveStock_0',249,'2015-06-20 06:42:35.772','2015-06-20 06:57:35.772','',0,null,'semaphore','172.16.185.22',null,null,0,0)
          /*!*/;
          # at 123700973
          #150620  6:42:35 server id 3  end_log_pos 123701000     Xid = 6680351941
          COMMIT/*!*/;
          # at 123701000

          Here comes the log output of the slave (including the part, where we started the slave again):

          150620  6:43:35 [ERROR] Slave SQL: Error 'Duplicate entry 'ReserveStock_0-249' for key 'identry_idqueue'' on query. Default database: 'optovision'. Query: 'insert into `queueentry`(`identry`,`idqueue`,`entrydate`,`removedate`,`info`,`prio`,`reserved`,`ObjectContext`,`OriginHost`,`ReservingHost`,`UnreserveDate`,`ReservationCounter`,`OptimisticLockField`)values('ReserveStock_0',249,'2015-06-20 06:42:35.772','2015-06-20 06:57:35.772','',0,null,'semaphore','172.16.185.22',null,null,0,0)', Gtid 0-3-506902041, Internal MariaDB error code: 1062
          150620  6:43:35 [Warning] Slave: Duplicate entry 'ReserveStock_0-249' for key 'identry_idqueue' Error_code: 1062
          150620  6:43:35 [ERROR] Error running query, slave SQL thread aborted. Fix the problem, and restart the slave SQL thread with "SLAVE START". We stopped at log 'mysql-bin.007837' position 123699757; GTID position '77-3-552,0-3-506902038,3-3-2,2-3-550224,17-3-180,1-3-7'
          150620  6:43:35 [Note] Slave SQL thread exiting, replication stopped in log 'mysql-bin.007837' at position 123700505; GTID position '77-3-552,0-3-506902040,3-3-2,2-3-550224,17-3-180,1-3-7'
          150620 13:29:52 [ERROR] Error reading packet from server: Lost connection to MySQL server during query ( server_errno=2013)
          150620 13:29:52 [Note] Slave I/O thread killed while reading event
          150620 13:29:52 [Note] Slave I/O thread exiting, read up to log 'mysql-bin.007837', position 584339931; GTID position 77-3-552,0-3-507350546,3-3-2,2-3-550224,1-3-7,17-3-180
          150620 13:30:39 [Note] Slave SQL thread initialized, starting replication in log 'mysql-bin.007837' at position 123700505, relay log './optosql5-relay-bin.000001' position: 4; GTID position '77-3-552,0-3-506902040,3-3-2,2-3-550224,17-3-180,1-3-7'
          150620 13:30:39 [Note] Slave I/O thread: connected to master 'repl@172.16.184.149:3306',replication starts at GTID position '77-3-552,0-3-506902040,3-3-2,2-3-550224,1-3-7,17-3-180'

          If it is of any importance, the slave reports that from the cid=6680351921 the delete statement was finished first (at least it is placed first in the slaves binlog). This was the case in all groups of transactions I analyzed. Of course this might be, because the delete is faster than the other statement. The binlog of the slave ends like this:

          #150620  6:42:35 server id 3  end_log_pos 113370402     GTID 0-3-506902039 cid=22219512
          /*!100001 SET @@session.gtid_seq_no=506902039*//*!*/;
          BEGIN
          /*!*/;
          # at 113370402
          #150620  6:42:35 server id 3  end_log_pos 113370554     Query   thread_id=19732490      exec_time=60
              error_code=0
          SET TIMESTAMP=1434775355/*!*/;
          delete from `queueentry` where ((`OID` = 10337810) and (`OptimisticLockField` = 0))
          /*!*/;
          # at 113370554
          #150620  6:42:35 server id 3  end_log_pos 113370581     Xid = 22219512
          COMMIT/*!*/;
          # at 113370581
          #150620  6:42:35 server id 3  end_log_pos 113370621     GTID 0-3-506902040 cid=22219512
          /*!100001 SET @@session.gtid_seq_no=506902040*//*!*/;
          BEGIN
          /*!*/;
          # at 113370621
          #150620  6:42:35 server id 3  end_log_pos 113371083     Query   thread_id=19732490      exec_time=60
              error_code=0
          SET TIMESTAMP=1434775355/*!*/;
          update `queueentry` set `identry`='637714',`idqueue`=294,`entrydate`='2015-06-20 00:59:46 ',`removedate`='2015-06-21 00:59:46 ',`info`='',`prio`=0,`reserved`='2015-06-20 06:42:35.756',`ObjectContext`='job',`OriginHost`='172.16.185.22',`ReservingHost`='172.16.185.21',`UnreserveDate`=null,`ReservationCounter`=2,`OptimisticLockField`=4 where ((`OID` = 10283585) and (`OptimisticLockField` = 3))
          /*!*/;
          # at 113371083
          #150620  6:42:35 server id 3  end_log_pos 113371110     Xid = 22219508
          COMMIT/*!*/;
          # at 113371110
          #150620  7:30:01 server id 5  end_log_pos 113371153     Rotate to mysql-bin.002721  pos: 4
          DELIMITER ;
          # End of log file
          ROLLBACK /* added by mysqlbinlog */;
          /*!50003 SET COMPLETION_TYPE=@OLD_COMPLETION_TYPE*/;
          /*!50530 SET @@SESSION.PSEUDO_SLAVE_MODE=0*/;

          But as you can see, even the slave says that those two transactions (delete and update) can be executed in parallel (they got the same cid on the slave).

          We did not update the master or set any variables beside binlog_commit_wait_count (during the crashes it was set to 4, now we are back to 0).

          As I wrote we now have a second slave which we can use for testing. But still it only happens with our production master (and as you can see from the timestamp, most of the time when we are not in the office). But I think we could do the following:

          • Configure our production slave (it is heavily used for statistical analysis and so on, so we need it to be up and running) to execute all transactions from one transaction group sequentially, while still executing transactions from out of band replication in parallel.
          • Keep the configuration of the test slave as it is, so that we can play around with the settings and/or executables for you. So if you want, we can try to run a debugging version here, or something else.

          That is all we have for the moment. How can we help to proceed?

          Best regards,

          Thomas Mischke

          thomas.mischke Thomas Mischke added a comment - - edited I am coming back with bad news. We used the weekend to update our slave to MariaDB 10.0.19, and it took just a few hours for the error to reappear. Then we set up a second slave with MariaDB 10.0.20. Again after a few hours we got the same error. Well, "the same" does not match it 100%... Here are the details for the error with the version 10.0.19: This time, the command "stop slave" worked, so at least something has changed, since in the old version, this command hung. During catching up the same error occurred another two times, each time "stop slave" worked. It is no longer the alphabetically last value in the index, that is causing the problem. This time it was a value from somewhere in the middle, but "always the same value". Again the slave was a few seconds behind the master, when the error happened, and no further transactions took place between those I write about. As before the last transactions to complete were two transactions grouped together. The new thing: The slave itself grouped them together in its binlog this time. This was not the case with the old version. To me this indicates that none of these transactions was locked, killed and reexecuted. This time the transaction to fail was itself a part of a group of transactions. All four cases that I took a look at had this pattern, that is every time it was part of a group of transactions, it was never a standalone transaction. Here is the binlog of the master. The two statements with cid=6680351921 were the last two to be executed on the slave. After that a new group of two transactions got to the slave. The "insert" statement was causing the crash. #150620 6:42:35 server id 3 end_log_pos 123699299 GTID 0-3-506902038 cid=6680351921 /*!100001 SET @@session.gtid_seq_no=506902038*//*!*/; BEGIN /*!*/; # at 123699299 #150620 6:42:35 server id 3 end_log_pos 123699327 Intvar SET INSERT_ID=10337814/*!*/; # at 123699327 #150620 6:42:35 server id 3 end_log_pos 123699730 Query thread_id=21184521 exec_time=0 error_code=0 SET TIMESTAMP=1434775355/*!*/; insert into `queueentry`(`identry`,`idqueue`,`entrydate`,`removedate`,`info`,`prio`,`reserved`,`Obje ctContext`,`OriginHost`,`ReservingHost`,`UnreserveDate`,`ReservationCounter`,`OptimisticLockField`)v alues('RemoveFromStock_0',249,'2015-06-20 06:42:35.725','2015-06-20 06:57:35.725','',0,null,'semapho re','172.16.185.21',null,null,0,0) /*!*/; # at 123699730 #150620 6:42:35 server id 3 end_log_pos 123699757 Xid = 6680351917 COMMIT/*!*/; # at 123699757 #150620 6:42:35 server id 3 end_log_pos 123699797 GTID 0-3-506902039 cid=6680351921 /*!100001 SET @@session.gtid_seq_no=506902039*//*!*/; BEGIN /*!*/; # at 123699797 #150620 6:42:35 server id 3 end_log_pos 123699949 Query thread_id=19732490 exec_time=0 error_code=0 SET TIMESTAMP=1434775355/*!*/; delete from `queueentry` where ((`OID` = 10337810) and (`OptimisticLockField` = 0)) /*!*/; # at 123699949 #150620 6:42:35 server id 3 end_log_pos 123699976 Xid = 6680351922 COMMIT/*!*/; # at 123699976 #150620 6:42:35 server id 3 end_log_pos 123700016 GTID 0-3-506902040 cid=6680351936 /*!100001 SET @@session.gtid_seq_no=506902040*//*!*/; BEGIN /*!*/; # at 123700016 #150620 6:42:35 server id 3 end_log_pos 123700478 Query thread_id=19732490 exec_time=0 error_code=0 SET TIMESTAMP=1434775355/*!*/; update `queueentry` set `identry`='637714',`idqueue`=294,`entrydate`='2015-06-20 00:59:46 ',`removedate`='2015-06-21 00:59:46 ',`info`='',`prio`=0,`reserved`='2015-06-20 06:42:35.756',`ObjectContext`='job',`OriginHost`='172.16.185.22',`ReservingHost`='172.16.185.21',`UnreserveDate`=null,`ReservationCounter`=2,`OptimisticLockField`=4 where ((`OID` = 10283585) and (`OptimisticLockField` = 3)) /*!*/; # at 123700478 #150620 6:42:35 server id 3 end_log_pos 123700505 Xid = 6680351933 COMMIT/*!*/; # at 123700505 #150620 6:42:35 server id 3 end_log_pos 123700545 GTID 0-3-506902041 cid=6680351936 /*!100001 SET @@session.gtid_seq_no=506902041*//*!*/; BEGIN /*!*/; # at 123700545 #150620 6:42:35 server id 3 end_log_pos 123700573 Intvar SET INSERT_ID=10337815/*!*/; # at 123700573 #150620 6:42:35 server id 3 end_log_pos 123700973 Query thread_id=21184521 exec_time=0 error_code=0 SET TIMESTAMP=1434775355/*!*/; insert into `queueentry`(`identry`,`idqueue`,`entrydate`,`removedate`,`info`,`prio`,`reserved`,`ObjectContext`,`OriginHost`,`ReservingHost`,`UnreserveDate`,`ReservationCounter`,`OptimisticLockField`)values('ReserveStock_0',249,'2015-06-20 06:42:35.772','2015-06-20 06:57:35.772','',0,null,'semaphore','172.16.185.22',null,null,0,0) /*!*/; # at 123700973 #150620 6:42:35 server id 3 end_log_pos 123701000 Xid = 6680351941 COMMIT/*!*/; # at 123701000 Here comes the log output of the slave (including the part, where we started the slave again): 150620 6:43:35 [ERROR] Slave SQL: Error 'Duplicate entry 'ReserveStock_0-249' for key 'identry_idqueue'' on query. Default database: 'optovision'. Query: 'insert into `queueentry`(`identry`,`idqueue`,`entrydate`,`removedate`,`info`,`prio`,`reserved`,`ObjectContext`,`OriginHost`,`ReservingHost`,`UnreserveDate`,`ReservationCounter`,`OptimisticLockField`)values('ReserveStock_0',249,'2015-06-20 06:42:35.772','2015-06-20 06:57:35.772','',0,null,'semaphore','172.16.185.22',null,null,0,0)', Gtid 0-3-506902041, Internal MariaDB error code: 1062 150620 6:43:35 [Warning] Slave: Duplicate entry 'ReserveStock_0-249' for key 'identry_idqueue' Error_code: 1062 150620 6:43:35 [ERROR] Error running query, slave SQL thread aborted. Fix the problem, and restart the slave SQL thread with "SLAVE START". We stopped at log 'mysql-bin.007837' position 123699757; GTID position '77-3-552,0-3-506902038,3-3-2,2-3-550224,17-3-180,1-3-7' 150620 6:43:35 [Note] Slave SQL thread exiting, replication stopped in log 'mysql-bin.007837' at position 123700505; GTID position '77-3-552,0-3-506902040,3-3-2,2-3-550224,17-3-180,1-3-7' 150620 13:29:52 [ERROR] Error reading packet from server: Lost connection to MySQL server during query ( server_errno=2013) 150620 13:29:52 [Note] Slave I/O thread killed while reading event 150620 13:29:52 [Note] Slave I/O thread exiting, read up to log 'mysql-bin.007837', position 584339931; GTID position 77-3-552,0-3-507350546,3-3-2,2-3-550224,1-3-7,17-3-180 150620 13:30:39 [Note] Slave SQL thread initialized, starting replication in log 'mysql-bin.007837' at position 123700505, relay log './optosql5-relay-bin.000001' position: 4; GTID position '77-3-552,0-3-506902040,3-3-2,2-3-550224,17-3-180,1-3-7' 150620 13:30:39 [Note] Slave I/O thread: connected to master 'repl@172.16.184.149:3306',replication starts at GTID position '77-3-552,0-3-506902040,3-3-2,2-3-550224,1-3-7,17-3-180' If it is of any importance, the slave reports that from the cid=6680351921 the delete statement was finished first (at least it is placed first in the slaves binlog). This was the case in all groups of transactions I analyzed. Of course this might be, because the delete is faster than the other statement. The binlog of the slave ends like this: #150620 6:42:35 server id 3 end_log_pos 113370402 GTID 0-3-506902039 cid=22219512 /*!100001 SET @@session.gtid_seq_no=506902039*//*!*/; BEGIN /*!*/; # at 113370402 #150620 6:42:35 server id 3 end_log_pos 113370554 Query thread_id=19732490 exec_time=60 error_code=0 SET TIMESTAMP=1434775355/*!*/; delete from `queueentry` where ((`OID` = 10337810) and (`OptimisticLockField` = 0)) /*!*/; # at 113370554 #150620 6:42:35 server id 3 end_log_pos 113370581 Xid = 22219512 COMMIT/*!*/; # at 113370581 #150620 6:42:35 server id 3 end_log_pos 113370621 GTID 0-3-506902040 cid=22219512 /*!100001 SET @@session.gtid_seq_no=506902040*//*!*/; BEGIN /*!*/; # at 113370621 #150620 6:42:35 server id 3 end_log_pos 113371083 Query thread_id=19732490 exec_time=60 error_code=0 SET TIMESTAMP=1434775355/*!*/; update `queueentry` set `identry`='637714',`idqueue`=294,`entrydate`='2015-06-20 00:59:46 ',`removedate`='2015-06-21 00:59:46 ',`info`='',`prio`=0,`reserved`='2015-06-20 06:42:35.756',`ObjectContext`='job',`OriginHost`='172.16.185.22',`ReservingHost`='172.16.185.21',`UnreserveDate`=null,`ReservationCounter`=2,`OptimisticLockField`=4 where ((`OID` = 10283585) and (`OptimisticLockField` = 3)) /*!*/; # at 113371083 #150620 6:42:35 server id 3 end_log_pos 113371110 Xid = 22219508 COMMIT/*!*/; # at 113371110 #150620 7:30:01 server id 5 end_log_pos 113371153 Rotate to mysql-bin.002721 pos: 4 DELIMITER ; # End of log file ROLLBACK /* added by mysqlbinlog */; /*!50003 SET COMPLETION_TYPE=@OLD_COMPLETION_TYPE*/; /*!50530 SET @@SESSION.PSEUDO_SLAVE_MODE=0*/; But as you can see, even the slave says that those two transactions (delete and update) can be executed in parallel (they got the same cid on the slave). We did not update the master or set any variables beside binlog_commit_wait_count (during the crashes it was set to 4, now we are back to 0). As I wrote we now have a second slave which we can use for testing. But still it only happens with our production master (and as you can see from the timestamp, most of the time when we are not in the office). But I think we could do the following: Configure our production slave (it is heavily used for statistical analysis and so on, so we need it to be up and running) to execute all transactions from one transaction group sequentially, while still executing transactions from out of band replication in parallel. Keep the configuration of the test slave as it is, so that we can play around with the settings and/or executables for you. So if you want, we can try to run a debugging version here, or something else. That is all we have for the moment. How can we help to proceed? Best regards, Thomas Mischke
          thomas.mischke Thomas Mischke made changes -
          Affects Version/s 10.0.20 [ 19201 ]
          Affects Version/s 10.0.19 [ 19200 ]
          elenst Elena Stepanova made changes -
          Assignee Elena Stepanova [ elenst ] Kristian Nielsen [ knielsen ]
          knielsen Kristian Nielsen made changes -
          Status Open [ 1 ] In Progress [ 3 ]

          Thanks for your effor to supply the additional information.

          So because of the differences you mention, it seems possible that this could
          be a different issue than the original.

          What we see is the following two groups on the master, each with two
          transactions:

          Group 1:

          INSERT1 of key ('RemoveFromStock_0',249)
          DELETE of (`OID` = 10337810) and (`OptimisticLockField` = 0)

          Group 2:

          UPDATE to key ('637714',294)
          INSERT2 of key ('ReserveStock_0',249)

          It is the INSERT2 that gets a duplicate key violation.
          I am guessing that the duplicate key is removed just before INSERT2, either
          by the DELETE or by the UPDATE. So I need to determine which of the two it
          is.

          Can you tell, from looking in the binlog, or from your knowledge of the
          application, or maybe from analysing the multiple different failures,
          whether it is the DELETE or the UPDATE that is removing the old row with the
          conflicting ('ReserveStock_0',249) unique key?

          I am wondering if it is the UPDATE. If so, you would certainly get the
          duplicate key error on the parallel slave, because UPDATE and INSERT2 are
          run in parallel. However, I cannot think of a way that the UPDATE could be
          put in the same group commit as the INSERT2 if they touched a common key
          value. Are you doing any special tricks with isolation levels or
          innodb_locks_unsafe_for_binlog or triggers or something? But even then I
          have a hard time thinking what it could be.

          If it is the DELETE that removes the key inserted by INSERT2, then the
          master binlog is correct (separate group commits). However, then the slave
          should not start the INSERT2 until the DELETE has started to commit, in
          which case again it is hard to see how InnoDB could report a duplicate key.

          Is there something more in group 1 before the INSERT1 and DELETE that could
          cause a (normal) conflict and a rollback-retry of the DELETE? Somehow
          causing INSERT2 to run before the DELETE? Though again I am not sure how
          that would be possible, it should be fixed in 10.0.20...

          In any case, it seems crucial to figure out which statement it is that is
          removing the duplicate key before the INSERT2, avoiding the duplicate key
          error on the master. Is it DELETE, or UPDATE (or something earlier in the
          binlog)?

          Also, can you confirm that in each case the error occured, just running
          START SLAVE (the slave was already stopped, right?) was enough to continue,
          the duplicate key error did not come again (at this position in the binlog,
          at least)? To rule out that this is an actual duplicate key error in the
          replication setup?

          knielsen Kristian Nielsen added a comment - Thanks for your effor to supply the additional information. So because of the differences you mention, it seems possible that this could be a different issue than the original. What we see is the following two groups on the master, each with two transactions: Group 1: INSERT1 of key ('RemoveFromStock_0',249) DELETE of (`OID` = 10337810) and (`OptimisticLockField` = 0) Group 2: UPDATE to key ('637714',294) INSERT2 of key ('ReserveStock_0',249) It is the INSERT2 that gets a duplicate key violation. I am guessing that the duplicate key is removed just before INSERT2, either by the DELETE or by the UPDATE. So I need to determine which of the two it is. Can you tell, from looking in the binlog, or from your knowledge of the application, or maybe from analysing the multiple different failures, whether it is the DELETE or the UPDATE that is removing the old row with the conflicting ('ReserveStock_0',249) unique key? I am wondering if it is the UPDATE. If so, you would certainly get the duplicate key error on the parallel slave, because UPDATE and INSERT2 are run in parallel. However, I cannot think of a way that the UPDATE could be put in the same group commit as the INSERT2 if they touched a common key value. Are you doing any special tricks with isolation levels or innodb_locks_unsafe_for_binlog or triggers or something? But even then I have a hard time thinking what it could be. If it is the DELETE that removes the key inserted by INSERT2, then the master binlog is correct (separate group commits). However, then the slave should not start the INSERT2 until the DELETE has started to commit, in which case again it is hard to see how InnoDB could report a duplicate key. Is there something more in group 1 before the INSERT1 and DELETE that could cause a (normal) conflict and a rollback-retry of the DELETE? Somehow causing INSERT2 to run before the DELETE? Though again I am not sure how that would be possible, it should be fixed in 10.0.20... In any case, it seems crucial to figure out which statement it is that is removing the duplicate key before the INSERT2, avoiding the duplicate key error on the master. Is it DELETE, or UPDATE (or something earlier in the binlog)? Also, can you confirm that in each case the error occured, just running START SLAVE (the slave was already stopped, right?) was enough to continue, the duplicate key error did not come again (at this position in the binlog, at least)? To rule out that this is an actual duplicate key error in the replication setup?

          The DELETE statement is removing the key, in all cases.

          I once again spoke to the admin doing the restarts at the weekend: It is as you say, the slave was already stopped, so a "START SLAVE" would have been enough (nevertheless he always typed "STOP SLAVE"). And yes, in all cases the slave continued without complaining about the same statement again (no "skip", manual delete or anything else required).

          For your other thoughts:

          • innodb_locks_unsafe_for_binlog is OFF
          • No triggers in the table involved
          • Nothing else special about this table (just "high traffic")
          • group1 and group2 are complete, that is no more statements in any of them

          All analysis I wrote above was on the production slave, that is with version 1.0.19. Currently I do not have access to the other machine with 10.0.20, so I am not 100% sure that the picture is the same there. I will check later and report in this ticket.

          thomas.mischke Thomas Mischke added a comment - The DELETE statement is removing the key, in all cases. I once again spoke to the admin doing the restarts at the weekend: It is as you say, the slave was already stopped, so a "START SLAVE" would have been enough (nevertheless he always typed "STOP SLAVE"). And yes, in all cases the slave continued without complaining about the same statement again (no "skip", manual delete or anything else required). For your other thoughts: innodb_locks_unsafe_for_binlog is OFF No triggers in the table involved Nothing else special about this table (just "high traffic") group1 and group2 are complete, that is no more statements in any of them All analysis I wrote above was on the production slave, that is with version 1.0.19. Currently I do not have access to the other machine with 10.0.20, so I am not 100% sure that the picture is the same there. I will check later and report in this ticket.

          And here is the result from version 10.0.20: To me it looks the same as with 10.0.19, but in case you find any differences:

          group 1:

          • INSERT 1
          • DELETE key X (via ID)

          group 2:

          • UPDATE
          • INSERT 2 (which fails)

          Error message of the slave:

          150621 19:08:03 [ERROR] Slave SQL: Error 'Duplicate entry 'ReserveStock_0-249' for key 'identry_idqueue'' on query. Default database: 'optovision'. Query: 'insert into `queueentry`(`identry`,`idqueue`,`entrydate`,`removedate`,`info`,`prio`,`reserved`,`ObjectContext`,`OriginHost`,`ReservingHost`,`UnreserveDate`,`ReservationCounter`,`OptimisticLockField`)values('ReserveStock_0',249,'2015-06-20 18:49:46.425','2015-06-20 19:04:46.425','',0,null,'semaphore','172.16.185.22',null,null,0,0)', Gtid 0-3-507561931, Internal MariaDB error code: 1062
          150621 19:08:03 [Warning] Slave: Duplicate entry 'ReserveStock_0-249' for key 'identry_idqueue' Error_code: 1062
          150621 19:08:03 [ERROR] Error running query, slave SQL thread aborted. Fix the problem, and restart the slave SQL thread with "SLAVE START". We stopped at log 'mysql-bin.007837' position 750622004; GTID position '77-3-552,0-3-507561928,3-3-2,2-3-550224,17-3-180,1-3-7'
          150621 19:08:03 [Note] Slave SQL thread exiting, replication stopped in log 'mysql-bin.007837' at position 750622755; GTID position '77-3-552,0-3-507561930,3-3-2,2-3-550224,17-3-180,1-3-7'
          150621 19:13:45 [Note] Slave SQL thread initialized, starting replication in log 'mysql-bin.007837' at position 750622755, relay log './OptoSQL3-relay-bin.000001' position: 4; GTID position '77-3-552,0-3-507561930,3-3-2,2-3-550224,17-3-180,1-3-7'

          And the binlog of the master at that position:

          #150620 18:49:46 server id 3  end_log_pos 750621546     GTID 0-3-507561928 cid=6696988508
          /*!100001 SET @@session.gtid_seq_no=507561928*//*!*/;
          BEGIN
          /*!*/;
          # at 750621546
          #150620 18:49:46 server id 3  end_log_pos 750621574     Intvar
          SET INSERT_ID=10429708/*!*/;
          # at 750621574
          #150620 18:49:46 server id 3  end_log_pos 750621977     Query   thread_id=19732488      exec_time=0     error_code=0
          SET TIMESTAMP=1434818986/*!*/;
          insert into `queueentry`(`identry`,`idqueue`,`entrydate`,`removedate`,`info`,`prio`,`reserved`,`ObjectContext`,`OriginHost`,`ReservingHost`,`UnreserveDate`,`ReservationCounter`,`OptimisticLockField`)values('
          RemoveFromStock_0',249,'2015-06-20 18:49:46.409','2015-06-20 19:04:46.409','',0,null,'semaphore','172.16.185.21',null,null,0,0)
          /*!*/;
          # at 750621977
          #150620 18:49:46 server id 3  end_log_pos 750622004     Xid = 6696988506
          COMMIT/*!*/;
          # at 750622004
          #150620 18:49:46 server id 3  end_log_pos 750622044     GTID 0-3-507561929 cid=6696988508
          /*!100001 SET @@session.gtid_seq_no=507561929*//*!*/;
          BEGIN
          /*!*/;
          # at 750622044
          #150620 18:49:46 server id 3  end_log_pos 750622196     Query   thread_id=19732489      exec_time=0     error_code=0
          SET TIMESTAMP=1434818986/*!*/;
          delete from `queueentry` where ((`OID` = 10429705) and (`OptimisticLockField` = 0))
          /*!*/;
          # at 750622196
          #150620 18:49:46 server id 3  end_log_pos 750622223     Xid = 6696988516
          COMMIT/*!*/;
          # at 750622223
          #150620 18:49:46 server id 3  end_log_pos 750622263     GTID 0-3-507561930 cid=6696988532
          /*!100001 SET @@session.gtid_seq_no=507561930*//*!*/;
          BEGIN
          /*!*/;
          # at 750622263
          #150620 18:49:46 server id 3  end_log_pos 750622728     Query   thread_id=19732488      exec_time=0     error_code=0
          SET TIMESTAMP=1434818986/*!*/;
          update `queueentry` set `identry`='605406',`idqueue`=294,`entrydate`='2015-06-19 23:44:00 ',`removedate`='2015-06-20 23:44:00 ',`info`='',`prio`=0,`reserved`='2015-06-20 18:49:46.425',`ObjectContext`='job',`
          OriginHost`='172.16.185.22',`ReservingHost`='172.16.185.21',`UnreserveDate`=null,`ReservationCounter`=14,`OptimisticLockField`=28 where ((`OID` = 10261579) and (`OptimisticLockField` = 27))
          /*!*/;
          # at 750622728
          #150620 18:49:46 server id 3  end_log_pos 750622755     Xid = 6696988530
          COMMIT/*!*/;
          # at 750622755
          #150620 18:49:46 server id 3  end_log_pos 750622795     GTID 0-3-507561931 cid=6696988532
          /*!100001 SET @@session.gtid_seq_no=507561931*//*!*/;
          BEGIN
          /*!*/;
          # at 750622795
          #150620 18:49:46 server id 3  end_log_pos 750622823     Intvar
          SET INSERT_ID=10429709/*!*/;
          # at 750622823
          #150620 18:49:46 server id 3  end_log_pos 750623223     Query   thread_id=19732489      exec_time=0     error_code=0
          SET TIMESTAMP=1434818986/*!*/;
          insert into `queueentry`(`identry`,`idqueue`,`entrydate`,`removedate`,`info`,`prio`,`reserved`,`ObjectContext`,`OriginHost`,`ReservingHost`,`UnreserveDate`,`ReservationCounter`,`OptimisticLockField`)values('ReserveStock_0',249,'2015-06-20 18:49:46.425','2015-06-20 19:04:46.425','',0,null,'semaphore','172.16.185.22',null,null,0,0)
          /*!*/;
          # at 750623223
          #150620 18:49:46 server id 3  end_log_pos 750623250     Xid = 6696988536
          COMMIT/*!*/;
          # at 750623250

          Again it was the same key that was hit during the error situation.

          thomas.mischke Thomas Mischke added a comment - And here is the result from version 10.0.20: To me it looks the same as with 10.0.19, but in case you find any differences: group 1: INSERT 1 DELETE key X (via ID) group 2: UPDATE INSERT 2 (which fails) Error message of the slave: 150621 19:08:03 [ERROR] Slave SQL: Error 'Duplicate entry 'ReserveStock_0-249' for key 'identry_idqueue'' on query. Default database: 'optovision'. Query: 'insert into `queueentry`(`identry`,`idqueue`,`entrydate`,`removedate`,`info`,`prio`,`reserved`,`ObjectContext`,`OriginHost`,`ReservingHost`,`UnreserveDate`,`ReservationCounter`,`OptimisticLockField`)values('ReserveStock_0',249,'2015-06-20 18:49:46.425','2015-06-20 19:04:46.425','',0,null,'semaphore','172.16.185.22',null,null,0,0)', Gtid 0-3-507561931, Internal MariaDB error code: 1062 150621 19:08:03 [Warning] Slave: Duplicate entry 'ReserveStock_0-249' for key 'identry_idqueue' Error_code: 1062 150621 19:08:03 [ERROR] Error running query, slave SQL thread aborted. Fix the problem, and restart the slave SQL thread with "SLAVE START". We stopped at log 'mysql-bin.007837' position 750622004; GTID position '77-3-552,0-3-507561928,3-3-2,2-3-550224,17-3-180,1-3-7' 150621 19:08:03 [Note] Slave SQL thread exiting, replication stopped in log 'mysql-bin.007837' at position 750622755; GTID position '77-3-552,0-3-507561930,3-3-2,2-3-550224,17-3-180,1-3-7' 150621 19:13:45 [Note] Slave SQL thread initialized, starting replication in log 'mysql-bin.007837' at position 750622755, relay log './OptoSQL3-relay-bin.000001' position: 4; GTID position '77-3-552,0-3-507561930,3-3-2,2-3-550224,17-3-180,1-3-7' And the binlog of the master at that position: #150620 18:49:46 server id 3 end_log_pos 750621546 GTID 0-3-507561928 cid=6696988508 /*!100001 SET @@session.gtid_seq_no=507561928*//*!*/; BEGIN /*!*/; # at 750621546 #150620 18:49:46 server id 3 end_log_pos 750621574 Intvar SET INSERT_ID=10429708/*!*/; # at 750621574 #150620 18:49:46 server id 3 end_log_pos 750621977 Query thread_id=19732488 exec_time=0 error_code=0 SET TIMESTAMP=1434818986/*!*/; insert into `queueentry`(`identry`,`idqueue`,`entrydate`,`removedate`,`info`,`prio`,`reserved`,`ObjectContext`,`OriginHost`,`ReservingHost`,`UnreserveDate`,`ReservationCounter`,`OptimisticLockField`)values(' RemoveFromStock_0',249,'2015-06-20 18:49:46.409','2015-06-20 19:04:46.409','',0,null,'semaphore','172.16.185.21',null,null,0,0) /*!*/; # at 750621977 #150620 18:49:46 server id 3 end_log_pos 750622004 Xid = 6696988506 COMMIT/*!*/; # at 750622004 #150620 18:49:46 server id 3 end_log_pos 750622044 GTID 0-3-507561929 cid=6696988508 /*!100001 SET @@session.gtid_seq_no=507561929*//*!*/; BEGIN /*!*/; # at 750622044 #150620 18:49:46 server id 3 end_log_pos 750622196 Query thread_id=19732489 exec_time=0 error_code=0 SET TIMESTAMP=1434818986/*!*/; delete from `queueentry` where ((`OID` = 10429705) and (`OptimisticLockField` = 0)) /*!*/; # at 750622196 #150620 18:49:46 server id 3 end_log_pos 750622223 Xid = 6696988516 COMMIT/*!*/; # at 750622223 #150620 18:49:46 server id 3 end_log_pos 750622263 GTID 0-3-507561930 cid=6696988532 /*!100001 SET @@session.gtid_seq_no=507561930*//*!*/; BEGIN /*!*/; # at 750622263 #150620 18:49:46 server id 3 end_log_pos 750622728 Query thread_id=19732488 exec_time=0 error_code=0 SET TIMESTAMP=1434818986/*!*/; update `queueentry` set `identry`='605406',`idqueue`=294,`entrydate`='2015-06-19 23:44:00 ',`removedate`='2015-06-20 23:44:00 ',`info`='',`prio`=0,`reserved`='2015-06-20 18:49:46.425',`ObjectContext`='job',` OriginHost`='172.16.185.22',`ReservingHost`='172.16.185.21',`UnreserveDate`=null,`ReservationCounter`=14,`OptimisticLockField`=28 where ((`OID` = 10261579) and (`OptimisticLockField` = 27)) /*!*/; # at 750622728 #150620 18:49:46 server id 3 end_log_pos 750622755 Xid = 6696988530 COMMIT/*!*/; # at 750622755 #150620 18:49:46 server id 3 end_log_pos 750622795 GTID 0-3-507561931 cid=6696988532 /*!100001 SET @@session.gtid_seq_no=507561931*//*!*/; BEGIN /*!*/; # at 750622795 #150620 18:49:46 server id 3 end_log_pos 750622823 Intvar SET INSERT_ID=10429709/*!*/; # at 750622823 #150620 18:49:46 server id 3 end_log_pos 750623223 Query thread_id=19732489 exec_time=0 error_code=0 SET TIMESTAMP=1434818986/*!*/; insert into `queueentry`(`identry`,`idqueue`,`entrydate`,`removedate`,`info`,`prio`,`reserved`,`ObjectContext`,`OriginHost`,`ReservingHost`,`UnreserveDate`,`ReservationCounter`,`OptimisticLockField`)values('ReserveStock_0',249,'2015-06-20 18:49:46.425','2015-06-20 19:04:46.425','',0,null,'semaphore','172.16.185.22',null,null,0,0) /*!*/; # at 750623223 #150620 18:49:46 server id 3 end_log_pos 750623250 Xid = 6696988536 COMMIT/*!*/; # at 750623250 Again it was the same key that was hit during the error situation.

          > As I wrote we now have a second slave which we can use for testing.

          > So if you want, we can try to run a debugging version here, or something
          > else.

          That would be most helpful.

          I will try to prepare a build for you with some extra debugging printouts
          that will trigger when the code hits the condition that I am guessing is
          responsible for the bug. I will let you know when it is ready.

          • Kristian.
          knielsen Kristian Nielsen added a comment - > As I wrote we now have a second slave which we can use for testing. > So if you want, we can try to run a debugging version here, or something > else. That would be most helpful. I will try to prepare a build for you with some extra debugging printouts that will trigger when the code hits the condition that I am guessing is responsible for the bug. I will let you know when it is ready. Kristian.

          Some notes on my current theory of what causes the bug. Maybe mostly for my
          own benefit, as it concerns deep details of the implementation, but then
          again maybe it will be of some interest to others anyway:

          What we see is that the INSERT2 of group 2 seems to run before before DELETE
          has committed. However, group 2 is not allowed to start before both the
          INSERT1 and DELETE of group 1 have started to execute their COMMIT.

          What can happen is that a deadlock was detected while DELETE was
          committing. This would cause DELETE to roll back, wait for INSERT1 to
          complete its commit, then be retried. If INSERT1 somehow starts to commit
          after DELETE has started to commit, but before DELETE is rolled back, group
          2 might think that all transactions in group 1 have started to commit.

          This is not supposed to be possible. But there were historically several
          bugs in this complex code, which could lead to this situation. It is
          possible that there is another similar bug, though I do not see what it
          would be at the moment.

          The other possibility is that the deadlock detected is actually a false
          alarm. Something like DELETE taking a lock that blocks INSERT1. This would
          cause a deadlock to be detected (because INSERT1 has to commit before
          DELETE). But if DELETE somehow releases the lock during executing, before
          committing, then there is no real deadlock. This would allow that a deadlock
          was signalled, then INSERT1 started to commit anyway, and only then was the
          deadlock signal detected by DELETE to cause a rollback and retry.

          Hopefully a debug build with extra printouts could help pinpoint which of
          these cases it is, and how the abnormal situation arises.

          knielsen Kristian Nielsen added a comment - Some notes on my current theory of what causes the bug. Maybe mostly for my own benefit, as it concerns deep details of the implementation, but then again maybe it will be of some interest to others anyway: What we see is that the INSERT2 of group 2 seems to run before before DELETE has committed. However, group 2 is not allowed to start before both the INSERT1 and DELETE of group 1 have started to execute their COMMIT. What can happen is that a deadlock was detected while DELETE was committing. This would cause DELETE to roll back, wait for INSERT1 to complete its commit, then be retried. If INSERT1 somehow starts to commit after DELETE has started to commit, but before DELETE is rolled back, group 2 might think that all transactions in group 1 have started to commit. This is not supposed to be possible. But there were historically several bugs in this complex code, which could lead to this situation. It is possible that there is another similar bug, though I do not see what it would be at the moment. The other possibility is that the deadlock detected is actually a false alarm. Something like DELETE taking a lock that blocks INSERT1. This would cause a deadlock to be detected (because INSERT1 has to commit before DELETE). But if DELETE somehow releases the lock during executing, before committing, then there is no real deadlock. This would allow that a deadlock was signalled, then INSERT1 started to commit anyway, and only then was the deadlock signal detected by DELETE to cause a rollback and retry. Hopefully a debug build with extra printouts could help pinpoint which of these cases it is, and how the abnormal situation arises.

          By the way, your observation with how the transactions were grouped on the slave are indeed interesting. As you say, they suggest that the DELETE did not conflict/deadlock with the UPDATE. The current theory is that the DELETE conflicts with the INSERT1.

          (as to why they would conflict, this is currently unknown. It seems to be related to some internal locking details of InnoDB/XtraDB. Maybe there are some special cases depending on exact BTree layout that occationally requires extra locking.)

          knielsen Kristian Nielsen added a comment - By the way, your observation with how the transactions were grouped on the slave are indeed interesting. As you say, they suggest that the DELETE did not conflict/deadlock with the UPDATE. The current theory is that the DELETE conflicts with the INSERT1. (as to why they would conflict, this is currently unknown. It seems to be related to some internal locking details of InnoDB/XtraDB. Maybe there are some special cases depending on exact BTree layout that occationally requires extra locking.)

          I have prepared a patch for a custom build of 10.0.20 with extra debugging
          information:

          http://lists.askmonty.org/pipermail/commits/2015-June/008104.html

          If you can apply this patch and compile, you can try it immediately, if not
          I am trying to get Buildbot fixed so that it can produce some custom
          binaries/packages for you to install.

          As you suggested, it would be really useful if you could run this build on
          your test slave and try to reproduce the problem.

          If you manage to reproduce it, there should be in the error log additional
          lines, all prefixed with "MDEV8302:". Those lines, along with details of the
          corresponding GTIDs from the binlog as you provided before, should provide
          valuable information about what the root cause of the issue might be.

          The patch also includes a change that might fix the issue completely,
          though I cannot be sure it will do so from the information currently
          available. So if this patch makes the issue no longer reproducible, that is
          also very valuable information. If this fix triggers, there should be a line
          (or lines) in the error log like this:

          MDEV8302: Skip mark_start_commit(GTID X-Y-Z) due to killed

          Again, any information in the error log and associated GTIDs will be most
          valuable in this case.

          Thanks,

          • Kristian.
          knielsen Kristian Nielsen added a comment - I have prepared a patch for a custom build of 10.0.20 with extra debugging information: http://lists.askmonty.org/pipermail/commits/2015-June/008104.html If you can apply this patch and compile, you can try it immediately, if not I am trying to get Buildbot fixed so that it can produce some custom binaries/packages for you to install. As you suggested, it would be really useful if you could run this build on your test slave and try to reproduce the problem. If you manage to reproduce it, there should be in the error log additional lines, all prefixed with "MDEV8302:". Those lines, along with details of the corresponding GTIDs from the binlog as you provided before, should provide valuable information about what the root cause of the issue might be. The patch also includes a change that might fix the issue completely, though I cannot be sure it will do so from the information currently available. So if this patch makes the issue no longer reproducible, that is also very valuable information. If this fix triggers, there should be a line (or lines) in the error log like this: MDEV8302: Skip mark_start_commit(GTID X-Y-Z) due to killed Again, any information in the error log and associated GTIDs will be most valuable in this case. Thanks, Kristian.

          I guess this is our chance to see, how easy it is to compile MariaDB from scratch

          We will give it a try. If we succeed we will let you know the results from the log file, if not, we will ask for binaries...

          thomas.mischke Thomas Mischke added a comment - I guess this is our chance to see, how easy it is to compile MariaDB from scratch We will give it a try. If we succeed we will let you know the results from the log file, if not, we will ask for binaries...

          The new version has been compiled and is running at the test machine. As soon as we have any log messages or other incidents...

          thomas.mischke Thomas Mischke added a comment - The new version has been compiled and is running at the test machine. As soon as we have any log messages or other incidents...

          We have first results from our logfile. The output after something more than 24 hours looks like this:

          MDEV8302: Deadlock kill GTID 0-3-533189420 (subid 533785258 in_commit=1) due to blocking GTID 0-3-533189419 (subid 533785257 in_commit=0)
          MDEV8302: Got deadlock kill in GTID 0-3-533189420 (subid 533785258 in_commit=1)
          MDEV8302: Retry #1 of GTID 0-3-533189420
           
          MDEV8302: Deadlock kill GTID 0-3-533798983 (subid 534401839 in_commit=1) due to blocking GTID 0-3-533798982 (subid 534401838 in_commit=0)
          MDEV8302: Got deadlock kill in GTID 0-3-533798983 (subid 534401839 in_commit=1)
          MDEV8302: Retry #1 of GTID 0-3-533798983
           
          MDEV8302: Deadlock kill GTID 0-3-533911175 (subid 534514031 in_commit=0) due to blocking GTID 0-3-533911174 (subid 534514030 in_commit=0)
          MDEV8302: Got deadlock kill in GTID 0-3-533911175 (subid 534514031 in_commit=1)
          MDEV8302: Retry #1 of GTID 0-3-533911175
           
          MDEV8302: Deadlock kill GTID 0-3-533992852 (subid 534595708 in_commit=0) due to blocking GTID 0-3-533992851 (subid 534595707 in_commit=0)
          MDEV8302: Skip mark_start_commit(GTID 0-3-533992852) due to killed
          MDEV8302: Got deadlock kill in GTID 0-3-533992852 (subid 534595708 in_commit=0)
          MDEV8302: Retry #1 of GTID 0-3-533992852
           
          MDEV8302: Deadlock kill GTID 0-3-535172063 (subid 535774919 in_commit=1) due to blocking GTID 0-3-535172062 (subid 535774918 in_commit=0)
          MDEV8302: Got deadlock kill in GTID 0-3-535172063 (subid 535774919 in_commit=1)
          MDEV8302: Retry #1 of GTID 0-3-535172063

          So from the logfiles it seems that we have three different cases that we met (the first two and the last seem equal from the logs, the others differ), but maybe the binlogs can tell more.

          Btw: Did I say that I like timestamps in log messages?

          The good news is that we did not have a replication breakdown since we started the new version of MariaDB as the slave. We will keep it running a few more days (or weeks on your request) to see, if any new error situations happen.

          Having said all this, here are the results from the binlog. Above there were 5 incidents, I will number them top down.

          Case 1) I took the freedom to include the transactions before and after the critical trasnactions as well into the output, since they involved the same table, so they might or might not be involved.

          #150630 13:14:25 server id 3  end_log_pos 826113262     GTID 0-3-533189418
          /*!100001 SET @@session.gtid_seq_no=533189418*//*!*/;
          BEGIN
          /*!*/;
          # at 826113262
          #150630 13:14:25 server id 3  end_log_pos 826113414     Query   thread_id=29682118      exec_time=0
               error_code=0
          SET TIMESTAMP=1435662865/*!*/;
          delete from `queueentry` where ((`OID` = 12625823) and (`OptimisticLockField` = 0))
          /*!*/;
          # at 826113414
          #150630 13:14:25 server id 3  end_log_pos 826113441     Xid = 7307759446
          COMMIT/*!*/;
          # at 826113441
          #150630 13:14:25 server id 3  end_log_pos 826113481     GTID 0-3-533189419 cid=7307759471
          /*!100001 SET @@session.gtid_seq_no=533189419*//*!*/;
          BEGIN
          /*!*/;
          # at 826113481
          #150630 13:14:25 server id 3  end_log_pos 826113509     Intvar
          SET INSERT_ID=12626044/*!*/;
          # at 826113509
          #150630 13:14:25 server id 3  end_log_pos 826113932     Query   thread_id=29682118      exec_time=0
               error_code=0
          SET TIMESTAMP=1435662865/*!*/;
          insert into `queueentry`(`identry`,`idqueue`,`entrydate`,`removedate`,`info`,`prio`,`reserved`,`ObjectContext`,`OriginHost`,`ReservingHost`,`UnreserveDate`,`ReservationCounter`,`OptimisticLockField`)values('SchedulerFromDataBase_Sched 7_DEBUG_0',249,'2015-06-30 13:14:25.762','2015-07-01 13:14:25.746','',0,null,'semaphore','172.16.184.22',null,null,0,0)
          /*!*/;
          # at 826113932
          #150630 13:14:25 server id 3  end_log_pos 826113959     Xid = 7307759467
          COMMIT/*!*/;
          # at 826113959
          #150630 13:14:25 server id 3  end_log_pos 826113999     GTID 0-3-533189420 cid=7307759471
          /*!100001 SET @@session.gtid_seq_no=533189420*//*!*/;
          BEGIN
          /*!*/;
          # at 826113999
          #150630 13:14:25 server id 3  end_log_pos 826114027     Intvar
          SET INSERT_ID=12626045/*!*/;
          # at 826114027
          #150630 13:14:25 server id 3  end_log_pos 826114436     Query   thread_id=29278186      exec_time=0
               error_code=0
          SET TIMESTAMP=1435662865/*!*/;
          insert into `queueentry`(`identry`,`idqueue`,`entrydate`,`removedate`,`info`,`prio`,`reserved`,`ObjectContext`,`OriginHost`,`ReservingHost`,`UnreserveDate`,`ReservationCounter`,`OptimisticLockField`)values('SemaphoreWebservice1_0',249,'2015-06-30 13:14:25.753','2015-06-30 14:14:25.753','',0,null,'semaphore','172.16.185.107',null,null,0,0)
          /*!*/;
          # at 826114436
          #150630 13:14:25 server id 3  end_log_pos 826114463     Xid = 7307759475
          COMMIT/*!*/;
          # at 826114463
          #150630 13:14:25 server id 3  end_log_pos 826114503     GTID 0-3-533189421 cid=7307759471
          /*!100001 SET @@session.gtid_seq_no=533189421*//*!*/;
          BEGIN
          /*!*/;
          # at 826114503
          #150630 13:14:25 server id 3  end_log_pos 826114655     Query   thread_id=29457136      exec_time=0
               error_code=0
          SET TIMESTAMP=1435662865/*!*/;
          delete from `queueentry` where ((`OID` = 12625824) and (`OptimisticLockField` = 0))
          /*!*/;
          # at 826114655
          #150630 13:14:25 server id 3  end_log_pos 826114682     Xid = 7307759482
          COMMIT/*!*/;
          # at 826114682
          #150630 13:14:25 server id 3  end_log_pos 826114720     GTID 0-3-533189422
          /*!100001 SET @@session.gtid_seq_no=533189422*//*!*/;
          BEGIN
          /*!*/;
          # at 826114720
          #150630 13:14:25 server id 3  end_log_pos 826114748     Intvar
          SET INSERT_ID=12626046/*!*/;
          # at 826114748
          #150630 13:14:25 server id 3  end_log_pos 826115171     Query   thread_id=29457136      exec_time=0
               error_code=0
          SET TIMESTAMP=1435662865/*!*/;
          insert into `queueentry`(`identry`,`idqueue`,`entrydate`,`removedate`,`info`,`prio`,`reserved`,`ObjectContext`,`OriginHost`,`ReservingHost`,`UnreserveDate`,`ReservationCounter`,`OptimisticLockField`)values('SchedulerFromDataBase_Sched 3_DEBUG_0',249,'2015-06-30 13:14:25.777','2015-07-01 13:14:25.762','',0,null,'semaphore','172.16.184.22',null,null,0,0)
          /*!*/;
          # at 826115171
          #150630 13:14:25 server id 3  end_log_pos 826115198     Xid = 7307759500
          COMMIT/*!*/;
          # at 826115198
          #150630 13:14:25 server id 3  end_log_pos 826115236     GTID 0-3-533189423
          /*!100001 SET @@session.gtid_seq_no=533189423*//*!*/;
          BEGIN
          /*!*/;
          # at 826115236
          #150630 13:14:25 server id 3  end_log_pos 826115388     Query   thread_id=29457136      exec_time=0
               error_code=0
          SET TIMESTAMP=1435662865/*!*/;
          delete from `queueentry` where ((`OID` = 12625825) and (`OptimisticLockField` = 0))
          /*!*/;
          # at 826115388
          #150630 13:14:25 server id 3  end_log_pos 826115415     Xid = 7307759519
          COMMIT/*!*/;
          # at 826115415

          So the pattern in this case:

          • group 1: delete
          • group 2: insert, insert and delete (second insert gets reported due to first)
          • group 3: insert
          • group 4: delete

          So this is a pattern we did not see so far when replication errors occurred.

          The slave itself devided group2 into two groups in its own binlog: The first insert goes into one group, the second insert and the delete in a second.

          Case 2) In this case no context before the critical transactions, since the transactions before took place in other tables.

          #150630 16:28:19 server id 3  end_log_pos 779744843     GTID 0-3-533798982 cid=7325035465
          /*!100001 SET @@session.gtid_seq_no=533798982*//*!*/;
          BEGIN
          /*!*/;
          # at 779744843
          #150630 16:28:19 server id 3  end_log_pos 779744871     Intvar
          SET INSERT_ID=12676580/*!*/;
          # at 779744871
          #150630 16:28:19 server id 3  end_log_pos 779745289     Query   thread_id=29456734      exec_time=0
               error_code=0
          SET TIMESTAMP=1435674499/*!*/;
          insert into `queueentry`(`identry`,`idqueue`,`entrydate`,`removedate`,`info`,`prio`,`reserved`,`ObjectC
          ontext`,`OriginHost`,`ReservingHost`,`UnreserveDate`,`ReservationCounter`,`OptimisticLockField`)values(
          'SchedulerFromDataBase_Sched 46_0',249,'2015-06-30 16:28:19.237','2015-07-01 16:28:19.221','',0,null,'s
          emaphore','172.16.185.21',null,null,0,0)
          /*!*/;
          # at 779745289
          #150630 16:28:19 server id 3  end_log_pos 779745316     Xid = 7325035461
          COMMIT/*!*/;
          # at 779745316
          #150630 16:28:19 server id 3  end_log_pos 779745356     GTID 0-3-533798983 cid=7325035465
          /*!100001 SET @@session.gtid_seq_no=533798983*//*!*/;
          BEGIN
          /*!*/;
          # at 779745356
          #150630 16:28:19 server id 3  end_log_pos 779745508     Query   thread_id=29456385      exec_time=0
               error_code=0
          SET TIMESTAMP=1435674499/*!*/;
          delete from `queueentry` where ((`OID` = 12676215) and (`OptimisticLockField` = 0))
          /*!*/;
          # at 779745508
          #150630 16:28:19 server id 3  end_log_pos 779745535     Xid = 7325035471
          COMMIT/*!*/;
          # at 779745535
          #150630 16:28:19 server id 3  end_log_pos 779745575     GTID 0-3-533798984 cid=7325035465
          /*!100001 SET @@session.gtid_seq_no=533798984*//*!*/;
          BEGIN
          /*!*/;
          # at 779745575
          #150630 16:28:19 server id 3  end_log_pos 779745603     Intvar
          SET INSERT_ID=12676581/*!*/;
          # at 779745603
          #150630 16:28:19 server id 3  end_log_pos 779746021     Query   thread_id=29278185      exec_time=0
               error_code=0
          SET TIMESTAMP=1435674499/*!*/;
          insert into `queueentry`(`identry`,`idqueue`,`entrydate`,`removedate`,`info`,`prio`,`reserved`,`ObjectContext`,`OriginHost`,`ReservingHost`,`UnreserveDate`,`ReservationCounter`,`OptimisticLockField`)values('SchedulerFromDataBase_Sched 51_0',249,'2015-06-30 16:28:19.237','2015-07-01 16:28:19.237','',0,null,'semaphore','172.16.185.21',null,null,0,0)
          /*!*/;
          # at 779746021
          #150630 16:28:19 server id 3  end_log_pos 779746048     Xid = 7325035482
          COMMIT/*!*/;
          # at 779746048
          #150630 16:28:19 server id 3  end_log_pos 779746088     GTID 0-3-533798985 cid=7325035512
          /*!100001 SET @@session.gtid_seq_no=533798985*//*!*/;
          BEGIN
          /*!*/;
          # at 779746088
          #150630 16:28:19 server id 3  end_log_pos 779746116     Intvar
          SET INSERT_ID=12676582/*!*/;
          # at 779746116
          #150630 16:28:19 server id 3  end_log_pos 779746534     Query   thread_id=29278185      exec_time=0
               error_code=0
          SET TIMESTAMP=1435674499/*!*/;
          insert into `queueentry`(`identry`,`idqueue`,`entrydate`,`removedate`,`info`,`prio`,`reserved`,`ObjectContext`,`OriginHost`,`ReservingHost`,`UnreserveDate`,`ReservationCounter`,`OptimisticLockField`)values('SchedulerFromDataBase_Sched 47_0',249,'2015-06-30 16:28:19.252','2015-07-01 16:28:19.252','',0,null,'semaphore','172.16.185.21',null,null,0,0)
          /*!*/;
          # at 779746534
          #150630 16:28:19 server id 3  end_log_pos 779746561     Xid = 7325035509
          COMMIT/*!*/;
          # at 779746561

          So the pattern this time:

          • group 1: insert, delete and insert (the delete being reported due to the first insert)
          • group 2: insert and a (not published) operation to another table

          The slave splitted the first group of transactions into two groups in its own binlog: insert and (delete and insert).

          Case 3) Again with some context before

          #150630 16:59:23 server id 3  end_log_pos 923478199     GTID 0-3-533911171 cid=7327914820
          /*!100001 SET @@session.gtid_seq_no=533911171*//*!*/;
          BEGIN
          /*!*/;
          # at 923478199
          #150630 16:59:23 server id 3  end_log_pos 923478351     Query   thread_id=29456357      exec_time=0
               error_code=0
          SET TIMESTAMP=1435676363/*!*/;
          delete from `queueentry` where ((`OID` = 12687426) and (`OptimisticLockField` = 0))
          /*!*/;
          # at 923478351
          #150630 16:59:23 server id 3  end_log_pos 923478378     Xid = 7327914849
          COMMIT/*!*/;
          # at 923478378
          #150630 16:59:23 server id 3  end_log_pos 923478418     GTID 0-3-533911172 cid=7327914820
          /*!100001 SET @@session.gtid_seq_no=533911172*//*!*/;
          BEGIN
          /*!*/;
          # at 923478418
          #150630 16:59:23 server id 3  end_log_pos 923478570     Query   thread_id=29278186      exec_time=0
               error_code=0
          SET TIMESTAMP=1435676363/*!*/;
          delete from `queueentry` where ((`OID` = 12687425) and (`OptimisticLockField` = 0))
          /*!*/;
          # at 923478570
          #150630 16:59:23 server id 3  end_log_pos 923478597     Xid = 7327914858
          COMMIT/*!*/;
          # at 923478597
          #150630 16:59:23 server id 3  end_log_pos 923478637     GTID 0-3-533911173 cid=7327914820
          /*!100001 SET @@session.gtid_seq_no=533911173*//*!*/;
          BEGIN
          /*!*/;
          # at 923478637
          #150630 16:59:23 server id 3  end_log_pos 923478789     Query   thread_id=29278185      exec_time=0
               error_code=0
          SET TIMESTAMP=1435676363/*!*/;
          delete from `queueentry` where ((`OID` = 12687424) and (`OptimisticLockField` = 0))
          /*!*/;
          # at 923478789
          #150630 16:59:23 server id 3  end_log_pos 923478816     Xid = 7327914869
          COMMIT/*!*/;
          # at 923478816
          #150630 16:59:23 server id 3  end_log_pos 923478856     GTID 0-3-533911174 cid=7327914911
          /*!100001 SET @@session.gtid_seq_no=533911174*//*!*/;
          BEGIN
          /*!*/;
          # at 923478856
          #150630 16:59:23 server id 3  end_log_pos 923478884     Intvar
          SET INSERT_ID=12687718/*!*/;
          # at 923478884
          #150630 16:59:23 server id 3  end_log_pos 923479302     Query   thread_id=29456385      exec_time=0
               error_code=0
          SET TIMESTAMP=1435676363/*!*/;
          insert into `queueentry`(`identry`,`idqueue`,`entrydate`,`removedate`,`info`,`prio`,`reserved`,`ObjectContext`,`OriginHost`,`ReservingHost`,`UnreserveDate`,`ReservationCounter`,`OptimisticLockField`)values('SchedulerFromDataBase_Sched 46_0',249,'2015-06-30 16:59:23.565','2015-07-01 16:59:23.565','',0,null,'semaphore','172.16.185.21',null,null,0,0)
          /*!*/;
          # at 923479302
          #150630 16:59:23 server id 3  end_log_pos 923479329     Xid = 7327914907
          COMMIT/*!*/;
          # at 923479329
          #150630 16:59:23 server id 3  end_log_pos 923479369     GTID 0-3-533911175 cid=7327914911
          /*!100001 SET @@session.gtid_seq_no=533911175*//*!*/;
          BEGIN
          /*!*/;
          # at 923479369
          #150630 16:59:23 server id 3  end_log_pos 923479397     Intvar
          SET INSERT_ID=12687719/*!*/;
          # at 923479397
          #150630 16:59:23 server id 3  end_log_pos 923479815     Query   thread_id=29278186      exec_time=0
               error_code=0
          SET TIMESTAMP=1435676363/*!*/;
          insert into `queueentry`(`identry`,`idqueue`,`entrydate`,`removedate`,`info`,`prio`,`reserved`,`ObjectContext`,`OriginHost`,`ReservingHost`,`UnreserveDate`,`ReservationCounter`,`OptimisticLockField`)values('SchedulerFromDataBase_Sched 47_0',249,'2015-06-30 16:59:23.565','2015-07-01 16:59:23.565','',0,null,'semaphore','172.16.185.21',null,null,0,0)
          /*!*/;
          # at 923479815
          #150630 16:59:23 server id 3  end_log_pos 923479842     Xid = 7327914916
          COMMIT/*!*/;
          # at 923479842
          #150630 16:59:23 server id 3  end_log_pos 923479882     GTID 0-3-533911176 cid=7327914911
          /*!100001 SET @@session.gtid_seq_no=533911176*//*!*/;
          BEGIN
          /*!*/;
          # at 923479882
          #150630 16:59:23 server id 3  end_log_pos 923479910     Intvar
          SET INSERT_ID=12687720/*!*/;
          # at 923479910
          #150630 16:59:23 server id 3  end_log_pos 923480328     Query   thread_id=29278185      exec_time=0
               error_code=0
          SET TIMESTAMP=1435676363/*!*/;
          insert into `queueentry`(`identry`,`idqueue`,`entrydate`,`removedate`,`info`,`prio`,`reserved`,`ObjectContext`,`OriginHost`,`ReservingHost`,`UnreserveDate`,`ReservationCounter`,`OptimisticLockField`)values('SchedulerFromDataBase_Sched 28_0',249,'2015-06-30 16:59:23.565','2015-07-01 16:59:23.565','',0,null,'semaphore','172.16.185.21',null,null,0,0)
          /*!*/;
          # at 923480328
          #150630 16:59:23 server id 3  end_log_pos 923480355     Xid = 7327914927
          COMMIT/*!*/;
          # at 923480355

          Pattern:

          • group 1: delete, delete and delete
          • group 2: insert, insert and insert (second insert gets reported due to the first)

          Again the slave splitted the group into two groups in its binlog.

          Case 4) Again with context, just to be sure... (only context before, since afterwards other tables are busy)

          #150630 17:23:26 server id 3  end_log_pos 1031374911    GTID 0-3-533992848 cid=7330164232
          /*!100001 SET @@session.gtid_seq_no=533992848*//*!*/;
          BEGIN
          /*!*/;
          # at 1031374911
          #150630 17:23:26 server id 3  end_log_pos 1031375063    Query   thread_id=29456357      exec_time=0
               error_code=0
          SET TIMESTAMP=1435677806/*!*/;
          delete from `queueentry` where ((`OID` = 12695140) and (`OptimisticLockField` = 0))
          /*!*/;
          # at 1031375063
          #150630 17:23:26 server id 3  end_log_pos 1031375090    Xid = 7330164230
          COMMIT/*!*/;
          # at 1031375090
          #150630 17:23:26 server id 3  end_log_pos 1031375130    GTID 0-3-533992849 cid=7330164232
          /*!100001 SET @@session.gtid_seq_no=533992849*//*!*/;
          BEGIN
          /*!*/;
          # at 1031375130
          #150630 17:23:26 server id 3  end_log_pos 1031375282    Query   thread_id=29278185      exec_time=0
               error_code=0
          SET TIMESTAMP=1435677806/*!*/;
          delete from `queueentry` where ((`OID` = 12695138) and (`OptimisticLockField` = 0))
          /*!*/;
          # at 1031375282
          #150630 17:23:26 server id 3  end_log_pos 1031375309    Xid = 7330164239
          COMMIT/*!*/;
          # at 1031375309
          #150630 17:23:26 server id 3  end_log_pos 1031375349    GTID 0-3-533992850 cid=7330164232
          /*!100001 SET @@session.gtid_seq_no=533992850*//*!*/;
          BEGIN
          /*!*/;
          # at 1031375349
          #150630 17:23:26 server id 3  end_log_pos 1031375377    Intvar
          SET INSERT_ID=12695364/*!*/;
          # at 1031375377
          #150630 17:23:26 server id 3  end_log_pos 1031375795    Query   thread_id=29456384      exec_time=0
               error_code=0
          SET TIMESTAMP=1435677806/*!*/;
          insert into `queueentry`(`identry`,`idqueue`,`entrydate`,`removedate`,`info`,`prio`,`reserved`,`ObjectC
          ontext`,`OriginHost`,`ReservingHost`,`UnreserveDate`,`ReservationCounter`,`OptimisticLockField`)values(
          'SchedulerFromDataBase_Sched 32_0',249,'2015-06-30 17:23:26.396','2015-07-01 17:23:26.396','',0,null,'s
          emaphore','172.16.185.21',null,null,0,0)
          /*!*/;
          # at 1031375795
          #150630 17:23:26 server id 3  end_log_pos 1031375822    Xid = 7330164240
          COMMIT/*!*/;
          # at 1031375822
          #150630 17:23:26 server id 3  end_log_pos 1031375862    GTID 0-3-533992851 cid=7330164280
          /*!100001 SET @@session.gtid_seq_no=533992851*//*!*/;
          BEGIN
          /*!*/;
          # at 1031375862
          #150630 17:23:26 server id 3  end_log_pos 1031375890    Intvar
          SET INSERT_ID=12695365/*!*/;
          # at 1031375890
          #150630 17:23:26 server id 3  end_log_pos 1031376308    Query   thread_id=29278186      exec_time=0
               error_code=0
          SET TIMESTAMP=1435677806/*!*/;
          insert into `queueentry`(`identry`,`idqueue`,`entrydate`,`removedate`,`info`,`prio`,`reserved`,`ObjectContext`,`OriginHost`,`ReservingHost`,`UnreserveDate`,`ReservationCounter`,`OptimisticLockField`)values('SchedulerFromDataBase_Sched 45_0',249,'2015-06-30 17:23:26.427','2015-07-01 17:23:26.427','',0,null,'semaphore','172.16.185.22',null,null,0,0)
          /*!*/;
          # at 1031376308
          #150630 17:23:26 server id 3  end_log_pos 1031376335    Xid = 7330164275
          COMMIT/*!*/;
          # at 1031376335
          #150630 17:23:26 server id 3  end_log_pos 1031376375    GTID 0-3-533992852 cid=7330164280
          /*!100001 SET @@session.gtid_seq_no=533992852*//*!*/;
          BEGIN
          /*!*/;
          # at 1031376375
          #150630 17:23:26 server id 3  end_log_pos 1031376403    Intvar
          SET INSERT_ID=12695366/*!*/;
          # at 1031376403
          #150630 17:23:26 server id 3  end_log_pos 1031376821    Query   thread_id=29456384      exec_time=0
               error_code=0
          SET TIMESTAMP=1435677806/*!*/;
          insert into `queueentry`(`identry`,`idqueue`,`entrydate`,`removedate`,`info`,`prio`,`reserved`,`ObjectContext`,`OriginHost`,`ReservingHost`,`UnreserveDate`,`ReservationCounter`,`OptimisticLockField`)values('SchedulerFromDataBase_Sched 46_0',249,'2015-06-30 17:23:26.427','2015-07-01 17:23:26.427','',0,null,'semaphore','172.16.185.22',null,null,0,0)
          /*!*/;
          # at 1031376821
          #150630 17:23:26 server id 3  end_log_pos 1031376848    Xid = 7330164278
          COMMIT/*!*/;
          # at 1031376848
          #

          Pattern:

          • group 1: delete, delete, insert
          • group 2: insert, insert (second insert gets reported due to the first)

          Again the slave created two groups for group 2 in its own binlog.

          Case 5) Again with context before, but none after.

          #150701  0:44:15 server id 3  end_log_pos 747684889     GTID 0-3-535172059 cid=7365441428
          /*!100001 SET @@session.gtid_seq_no=535172059*//*!*/;
          BEGIN
          /*!*/;
          # at 747684889
          #150701  0:44:15 server id 3  end_log_pos 747684917     Intvar
          SET INSERT_ID=1191462/*!*/;
          # at 747684917
          #150701  0:44:15 server id 3  end_log_pos 747685235     Query   thread_id=30824422      exec_time=0
               error_code=0
          SET TIMESTAMP=1435704255/*!*/;
          insert into `abrsd01`(`ABZO05`,`ABZO01`,`ABATID`,`ABSRCE`,`ABBUID`,`ABORID`,`ABOPID`,`ABQRES`,`ABUPUS`,
          `ABUPBS`,`ABUPDT`,`ABUPTM`,`ABSTAT`,`OptimisticLockField`)values(55298,1,null,'CORD01',0,56578,2107742,
          1,'','OPTOSERVICE01',20150701,4414,'NEW',0)
          /*!*/;
          # at 747685235
          #150701  0:44:15 server id 3  end_log_pos 747685262     Xid = 7365441422
          COMMIT/*!*/;
          # at 747685262
          #150701  0:44:15 server id 3  end_log_pos 747685302     GTID 0-3-535172060 cid=7365441428
          /*!100001 SET @@session.gtid_seq_no=535172060*//*!*/;
          BEGIN
          /*!*/;
          # at 747685302
          #150701  0:44:15 server id 3  end_log_pos 747685746     Query   thread_id=29456384      exec_time=0
               error_code=0
          SET TIMESTAMP=1435704255/*!*/;
          update `queueentry` set `identry`='057209',`idqueue`=284,`entrydate`='2015-06-30 17:27:46 ',`removedate
          `='2015-07-01 17:27:46 ',`info`='RB* (Standard)',`prio`=0,`reserved`=null,`ObjectContext`='job',`Origin
          Host`='172.16.185.22',`ReservingHost`=null,`UnreserveDate`=null,`ReservationCounter`=2,`OptimisticLockF
          ield`=4 where ((`OID` = 12696865) and (`OptimisticLockField` = 3))
          /*!*/;
          # at 747685746
          #150701  0:44:15 server id 3  end_log_pos 747685773     Xid = 7365441437
          COMMIT/*!*/;
          # at 747685773
          #150701  0:44:15 server id 3  end_log_pos 747685813     GTID 0-3-535172061 cid=7365441428
          /*!100001 SET @@session.gtid_seq_no=535172061*//*!*/;
          BEGIN
          /*!*/;
          # at 747685813
          #150701  0:44:15 server id 3  end_log_pos 747685965     Query   thread_id=29456385      exec_time=0
               error_code=0
          SET TIMESTAMP=1435704255/*!*/;
          delete from `queueentry` where ((`OID` = 12794632) and (`OptimisticLockField` = 0))
          /*!*/;
          # at 747685965
          #150701  0:44:15 server id 3  end_log_pos 747685992     Xid = 7365441442
          COMMIT/*!*/;
          # at 747685992
          #150701  0:44:15 server id 3  end_log_pos 747686032     GTID 0-3-535172062 cid=7365441467
          /*!100001 SET @@session.gtid_seq_no=535172062*//*!*/;
          BEGIN
          /*!*/;
          # at 747686032
          #150701  0:44:15 server id 3  end_log_pos 747686060     Intvar
          SET INSERT_ID=12794638/*!*/;
          # at 747686060
          #150701  0:44:15 server id 3  end_log_pos 747686462     Query   thread_id=29456384      exec_time=0
               error_code=0
          SET TIMESTAMP=1435704255/*!*/;
          insert into `queueentry`(`identry`,`idqueue`,`entrydate`,`removedate`,`info`,`prio`,`reserved`,`ObjectContext`,`OriginHost`,`ReservingHost`,`UnreserveDate`,`ReservationCounter`,`OptimisticLockField`)values('RemoveFromStock_0',249,'2015-07-01 00:44:14.955','2015-07-01 00:59:14.955','',0,null,'semaphore','172.16.184.9',null,null,0,0)
          /*!*/;
          # at 747686462
          #150701  0:44:15 server id 3  end_log_pos 747686489     Xid = 7365441465
          COMMIT/*!*/;
          # at 747686489
          #150701  0:44:15 server id 3  end_log_pos 747686529     GTID 0-3-535172063 cid=7365441467
          /*!100001 SET @@session.gtid_seq_no=535172063*//*!*/;
          BEGIN
          /*!*/;
          # at 747686529
          #150701  0:44:15 server id 3  end_log_pos 747686681     Query   thread_id=29357465      exec_time=0
               error_code=0
          SET TIMESTAMP=1435704255/*!*/;
          delete from `queueentry` where ((`OID` = 12794633) and (`OptimisticLockField` = 0))
          /*!*/;
          # at 747686681
          #150701  0:44:15 server id 3  end_log_pos 747686708     Xid = 7365441473
          COMMIT/*!*/;
          # at 747686708

          Pattern:

          • group 1: insert, update, delete
          • group 2: insert, delete (the delete gets reported due to the insert)

          I did not check the slaves binlog here.

          Now having seen these logs, I must agree, this seems to be a different error than we reported first. It really seems to happen inside a group of transactions (while the other involved two groups of transactions). Another indicator for this is that the new errors happened while the slave was in sync with the master, while the other scenario happened when the slave was behind.

          Do you need anything more? Waiting for your comments...

          thomas.mischke Thomas Mischke added a comment - We have first results from our logfile. The output after something more than 24 hours looks like this: MDEV8302: Deadlock kill GTID 0-3-533189420 (subid 533785258 in_commit=1) due to blocking GTID 0-3-533189419 (subid 533785257 in_commit=0) MDEV8302: Got deadlock kill in GTID 0-3-533189420 (subid 533785258 in_commit=1) MDEV8302: Retry #1 of GTID 0-3-533189420   MDEV8302: Deadlock kill GTID 0-3-533798983 (subid 534401839 in_commit=1) due to blocking GTID 0-3-533798982 (subid 534401838 in_commit=0) MDEV8302: Got deadlock kill in GTID 0-3-533798983 (subid 534401839 in_commit=1) MDEV8302: Retry #1 of GTID 0-3-533798983   MDEV8302: Deadlock kill GTID 0-3-533911175 (subid 534514031 in_commit=0) due to blocking GTID 0-3-533911174 (subid 534514030 in_commit=0) MDEV8302: Got deadlock kill in GTID 0-3-533911175 (subid 534514031 in_commit=1) MDEV8302: Retry #1 of GTID 0-3-533911175   MDEV8302: Deadlock kill GTID 0-3-533992852 (subid 534595708 in_commit=0) due to blocking GTID 0-3-533992851 (subid 534595707 in_commit=0) MDEV8302: Skip mark_start_commit(GTID 0-3-533992852) due to killed MDEV8302: Got deadlock kill in GTID 0-3-533992852 (subid 534595708 in_commit=0) MDEV8302: Retry #1 of GTID 0-3-533992852   MDEV8302: Deadlock kill GTID 0-3-535172063 (subid 535774919 in_commit=1) due to blocking GTID 0-3-535172062 (subid 535774918 in_commit=0) MDEV8302: Got deadlock kill in GTID 0-3-535172063 (subid 535774919 in_commit=1) MDEV8302: Retry #1 of GTID 0-3-535172063 So from the logfiles it seems that we have three different cases that we met (the first two and the last seem equal from the logs, the others differ), but maybe the binlogs can tell more. Btw: Did I say that I like timestamps in log messages? The good news is that we did not have a replication breakdown since we started the new version of MariaDB as the slave. We will keep it running a few more days (or weeks on your request) to see, if any new error situations happen. Having said all this, here are the results from the binlog. Above there were 5 incidents, I will number them top down. Case 1) I took the freedom to include the transactions before and after the critical trasnactions as well into the output, since they involved the same table, so they might or might not be involved. #150630 13:14:25 server id 3 end_log_pos 826113262 GTID 0-3-533189418 /*!100001 SET @@session.gtid_seq_no=533189418*//*!*/; BEGIN /*!*/; # at 826113262 #150630 13:14:25 server id 3 end_log_pos 826113414 Query thread_id=29682118 exec_time=0 error_code=0 SET TIMESTAMP=1435662865/*!*/; delete from `queueentry` where ((`OID` = 12625823) and (`OptimisticLockField` = 0)) /*!*/; # at 826113414 #150630 13:14:25 server id 3 end_log_pos 826113441 Xid = 7307759446 COMMIT/*!*/; # at 826113441 #150630 13:14:25 server id 3 end_log_pos 826113481 GTID 0-3-533189419 cid=7307759471 /*!100001 SET @@session.gtid_seq_no=533189419*//*!*/; BEGIN /*!*/; # at 826113481 #150630 13:14:25 server id 3 end_log_pos 826113509 Intvar SET INSERT_ID=12626044/*!*/; # at 826113509 #150630 13:14:25 server id 3 end_log_pos 826113932 Query thread_id=29682118 exec_time=0 error_code=0 SET TIMESTAMP=1435662865/*!*/; insert into `queueentry`(`identry`,`idqueue`,`entrydate`,`removedate`,`info`,`prio`,`reserved`,`ObjectContext`,`OriginHost`,`ReservingHost`,`UnreserveDate`,`ReservationCounter`,`OptimisticLockField`)values('SchedulerFromDataBase_Sched 7_DEBUG_0',249,'2015-06-30 13:14:25.762','2015-07-01 13:14:25.746','',0,null,'semaphore','172.16.184.22',null,null,0,0) /*!*/; # at 826113932 #150630 13:14:25 server id 3 end_log_pos 826113959 Xid = 7307759467 COMMIT/*!*/; # at 826113959 #150630 13:14:25 server id 3 end_log_pos 826113999 GTID 0-3-533189420 cid=7307759471 /*!100001 SET @@session.gtid_seq_no=533189420*//*!*/; BEGIN /*!*/; # at 826113999 #150630 13:14:25 server id 3 end_log_pos 826114027 Intvar SET INSERT_ID=12626045/*!*/; # at 826114027 #150630 13:14:25 server id 3 end_log_pos 826114436 Query thread_id=29278186 exec_time=0 error_code=0 SET TIMESTAMP=1435662865/*!*/; insert into `queueentry`(`identry`,`idqueue`,`entrydate`,`removedate`,`info`,`prio`,`reserved`,`ObjectContext`,`OriginHost`,`ReservingHost`,`UnreserveDate`,`ReservationCounter`,`OptimisticLockField`)values('SemaphoreWebservice1_0',249,'2015-06-30 13:14:25.753','2015-06-30 14:14:25.753','',0,null,'semaphore','172.16.185.107',null,null,0,0) /*!*/; # at 826114436 #150630 13:14:25 server id 3 end_log_pos 826114463 Xid = 7307759475 COMMIT/*!*/; # at 826114463 #150630 13:14:25 server id 3 end_log_pos 826114503 GTID 0-3-533189421 cid=7307759471 /*!100001 SET @@session.gtid_seq_no=533189421*//*!*/; BEGIN /*!*/; # at 826114503 #150630 13:14:25 server id 3 end_log_pos 826114655 Query thread_id=29457136 exec_time=0 error_code=0 SET TIMESTAMP=1435662865/*!*/; delete from `queueentry` where ((`OID` = 12625824) and (`OptimisticLockField` = 0)) /*!*/; # at 826114655 #150630 13:14:25 server id 3 end_log_pos 826114682 Xid = 7307759482 COMMIT/*!*/; # at 826114682 #150630 13:14:25 server id 3 end_log_pos 826114720 GTID 0-3-533189422 /*!100001 SET @@session.gtid_seq_no=533189422*//*!*/; BEGIN /*!*/; # at 826114720 #150630 13:14:25 server id 3 end_log_pos 826114748 Intvar SET INSERT_ID=12626046/*!*/; # at 826114748 #150630 13:14:25 server id 3 end_log_pos 826115171 Query thread_id=29457136 exec_time=0 error_code=0 SET TIMESTAMP=1435662865/*!*/; insert into `queueentry`(`identry`,`idqueue`,`entrydate`,`removedate`,`info`,`prio`,`reserved`,`ObjectContext`,`OriginHost`,`ReservingHost`,`UnreserveDate`,`ReservationCounter`,`OptimisticLockField`)values('SchedulerFromDataBase_Sched 3_DEBUG_0',249,'2015-06-30 13:14:25.777','2015-07-01 13:14:25.762','',0,null,'semaphore','172.16.184.22',null,null,0,0) /*!*/; # at 826115171 #150630 13:14:25 server id 3 end_log_pos 826115198 Xid = 7307759500 COMMIT/*!*/; # at 826115198 #150630 13:14:25 server id 3 end_log_pos 826115236 GTID 0-3-533189423 /*!100001 SET @@session.gtid_seq_no=533189423*//*!*/; BEGIN /*!*/; # at 826115236 #150630 13:14:25 server id 3 end_log_pos 826115388 Query thread_id=29457136 exec_time=0 error_code=0 SET TIMESTAMP=1435662865/*!*/; delete from `queueentry` where ((`OID` = 12625825) and (`OptimisticLockField` = 0)) /*!*/; # at 826115388 #150630 13:14:25 server id 3 end_log_pos 826115415 Xid = 7307759519 COMMIT/*!*/; # at 826115415 So the pattern in this case: group 1: delete group 2: insert, insert and delete (second insert gets reported due to first) group 3: insert group 4: delete So this is a pattern we did not see so far when replication errors occurred. The slave itself devided group2 into two groups in its own binlog: The first insert goes into one group, the second insert and the delete in a second. Case 2) In this case no context before the critical transactions, since the transactions before took place in other tables. #150630 16:28:19 server id 3 end_log_pos 779744843 GTID 0-3-533798982 cid=7325035465 /*!100001 SET @@session.gtid_seq_no=533798982*//*!*/; BEGIN /*!*/; # at 779744843 #150630 16:28:19 server id 3 end_log_pos 779744871 Intvar SET INSERT_ID=12676580/*!*/; # at 779744871 #150630 16:28:19 server id 3 end_log_pos 779745289 Query thread_id=29456734 exec_time=0 error_code=0 SET TIMESTAMP=1435674499/*!*/; insert into `queueentry`(`identry`,`idqueue`,`entrydate`,`removedate`,`info`,`prio`,`reserved`,`ObjectC ontext`,`OriginHost`,`ReservingHost`,`UnreserveDate`,`ReservationCounter`,`OptimisticLockField`)values( 'SchedulerFromDataBase_Sched 46_0',249,'2015-06-30 16:28:19.237','2015-07-01 16:28:19.221','',0,null,'s emaphore','172.16.185.21',null,null,0,0) /*!*/; # at 779745289 #150630 16:28:19 server id 3 end_log_pos 779745316 Xid = 7325035461 COMMIT/*!*/; # at 779745316 #150630 16:28:19 server id 3 end_log_pos 779745356 GTID 0-3-533798983 cid=7325035465 /*!100001 SET @@session.gtid_seq_no=533798983*//*!*/; BEGIN /*!*/; # at 779745356 #150630 16:28:19 server id 3 end_log_pos 779745508 Query thread_id=29456385 exec_time=0 error_code=0 SET TIMESTAMP=1435674499/*!*/; delete from `queueentry` where ((`OID` = 12676215) and (`OptimisticLockField` = 0)) /*!*/; # at 779745508 #150630 16:28:19 server id 3 end_log_pos 779745535 Xid = 7325035471 COMMIT/*!*/; # at 779745535 #150630 16:28:19 server id 3 end_log_pos 779745575 GTID 0-3-533798984 cid=7325035465 /*!100001 SET @@session.gtid_seq_no=533798984*//*!*/; BEGIN /*!*/; # at 779745575 #150630 16:28:19 server id 3 end_log_pos 779745603 Intvar SET INSERT_ID=12676581/*!*/; # at 779745603 #150630 16:28:19 server id 3 end_log_pos 779746021 Query thread_id=29278185 exec_time=0 error_code=0 SET TIMESTAMP=1435674499/*!*/; insert into `queueentry`(`identry`,`idqueue`,`entrydate`,`removedate`,`info`,`prio`,`reserved`,`ObjectContext`,`OriginHost`,`ReservingHost`,`UnreserveDate`,`ReservationCounter`,`OptimisticLockField`)values('SchedulerFromDataBase_Sched 51_0',249,'2015-06-30 16:28:19.237','2015-07-01 16:28:19.237','',0,null,'semaphore','172.16.185.21',null,null,0,0) /*!*/; # at 779746021 #150630 16:28:19 server id 3 end_log_pos 779746048 Xid = 7325035482 COMMIT/*!*/; # at 779746048 #150630 16:28:19 server id 3 end_log_pos 779746088 GTID 0-3-533798985 cid=7325035512 /*!100001 SET @@session.gtid_seq_no=533798985*//*!*/; BEGIN /*!*/; # at 779746088 #150630 16:28:19 server id 3 end_log_pos 779746116 Intvar SET INSERT_ID=12676582/*!*/; # at 779746116 #150630 16:28:19 server id 3 end_log_pos 779746534 Query thread_id=29278185 exec_time=0 error_code=0 SET TIMESTAMP=1435674499/*!*/; insert into `queueentry`(`identry`,`idqueue`,`entrydate`,`removedate`,`info`,`prio`,`reserved`,`ObjectContext`,`OriginHost`,`ReservingHost`,`UnreserveDate`,`ReservationCounter`,`OptimisticLockField`)values('SchedulerFromDataBase_Sched 47_0',249,'2015-06-30 16:28:19.252','2015-07-01 16:28:19.252','',0,null,'semaphore','172.16.185.21',null,null,0,0) /*!*/; # at 779746534 #150630 16:28:19 server id 3 end_log_pos 779746561 Xid = 7325035509 COMMIT/*!*/; # at 779746561 So the pattern this time: group 1: insert, delete and insert (the delete being reported due to the first insert) group 2: insert and a (not published) operation to another table The slave splitted the first group of transactions into two groups in its own binlog: insert and (delete and insert). Case 3) Again with some context before #150630 16:59:23 server id 3 end_log_pos 923478199 GTID 0-3-533911171 cid=7327914820 /*!100001 SET @@session.gtid_seq_no=533911171*//*!*/; BEGIN /*!*/; # at 923478199 #150630 16:59:23 server id 3 end_log_pos 923478351 Query thread_id=29456357 exec_time=0 error_code=0 SET TIMESTAMP=1435676363/*!*/; delete from `queueentry` where ((`OID` = 12687426) and (`OptimisticLockField` = 0)) /*!*/; # at 923478351 #150630 16:59:23 server id 3 end_log_pos 923478378 Xid = 7327914849 COMMIT/*!*/; # at 923478378 #150630 16:59:23 server id 3 end_log_pos 923478418 GTID 0-3-533911172 cid=7327914820 /*!100001 SET @@session.gtid_seq_no=533911172*//*!*/; BEGIN /*!*/; # at 923478418 #150630 16:59:23 server id 3 end_log_pos 923478570 Query thread_id=29278186 exec_time=0 error_code=0 SET TIMESTAMP=1435676363/*!*/; delete from `queueentry` where ((`OID` = 12687425) and (`OptimisticLockField` = 0)) /*!*/; # at 923478570 #150630 16:59:23 server id 3 end_log_pos 923478597 Xid = 7327914858 COMMIT/*!*/; # at 923478597 #150630 16:59:23 server id 3 end_log_pos 923478637 GTID 0-3-533911173 cid=7327914820 /*!100001 SET @@session.gtid_seq_no=533911173*//*!*/; BEGIN /*!*/; # at 923478637 #150630 16:59:23 server id 3 end_log_pos 923478789 Query thread_id=29278185 exec_time=0 error_code=0 SET TIMESTAMP=1435676363/*!*/; delete from `queueentry` where ((`OID` = 12687424) and (`OptimisticLockField` = 0)) /*!*/; # at 923478789 #150630 16:59:23 server id 3 end_log_pos 923478816 Xid = 7327914869 COMMIT/*!*/; # at 923478816 #150630 16:59:23 server id 3 end_log_pos 923478856 GTID 0-3-533911174 cid=7327914911 /*!100001 SET @@session.gtid_seq_no=533911174*//*!*/; BEGIN /*!*/; # at 923478856 #150630 16:59:23 server id 3 end_log_pos 923478884 Intvar SET INSERT_ID=12687718/*!*/; # at 923478884 #150630 16:59:23 server id 3 end_log_pos 923479302 Query thread_id=29456385 exec_time=0 error_code=0 SET TIMESTAMP=1435676363/*!*/; insert into `queueentry`(`identry`,`idqueue`,`entrydate`,`removedate`,`info`,`prio`,`reserved`,`ObjectContext`,`OriginHost`,`ReservingHost`,`UnreserveDate`,`ReservationCounter`,`OptimisticLockField`)values('SchedulerFromDataBase_Sched 46_0',249,'2015-06-30 16:59:23.565','2015-07-01 16:59:23.565','',0,null,'semaphore','172.16.185.21',null,null,0,0) /*!*/; # at 923479302 #150630 16:59:23 server id 3 end_log_pos 923479329 Xid = 7327914907 COMMIT/*!*/; # at 923479329 #150630 16:59:23 server id 3 end_log_pos 923479369 GTID 0-3-533911175 cid=7327914911 /*!100001 SET @@session.gtid_seq_no=533911175*//*!*/; BEGIN /*!*/; # at 923479369 #150630 16:59:23 server id 3 end_log_pos 923479397 Intvar SET INSERT_ID=12687719/*!*/; # at 923479397 #150630 16:59:23 server id 3 end_log_pos 923479815 Query thread_id=29278186 exec_time=0 error_code=0 SET TIMESTAMP=1435676363/*!*/; insert into `queueentry`(`identry`,`idqueue`,`entrydate`,`removedate`,`info`,`prio`,`reserved`,`ObjectContext`,`OriginHost`,`ReservingHost`,`UnreserveDate`,`ReservationCounter`,`OptimisticLockField`)values('SchedulerFromDataBase_Sched 47_0',249,'2015-06-30 16:59:23.565','2015-07-01 16:59:23.565','',0,null,'semaphore','172.16.185.21',null,null,0,0) /*!*/; # at 923479815 #150630 16:59:23 server id 3 end_log_pos 923479842 Xid = 7327914916 COMMIT/*!*/; # at 923479842 #150630 16:59:23 server id 3 end_log_pos 923479882 GTID 0-3-533911176 cid=7327914911 /*!100001 SET @@session.gtid_seq_no=533911176*//*!*/; BEGIN /*!*/; # at 923479882 #150630 16:59:23 server id 3 end_log_pos 923479910 Intvar SET INSERT_ID=12687720/*!*/; # at 923479910 #150630 16:59:23 server id 3 end_log_pos 923480328 Query thread_id=29278185 exec_time=0 error_code=0 SET TIMESTAMP=1435676363/*!*/; insert into `queueentry`(`identry`,`idqueue`,`entrydate`,`removedate`,`info`,`prio`,`reserved`,`ObjectContext`,`OriginHost`,`ReservingHost`,`UnreserveDate`,`ReservationCounter`,`OptimisticLockField`)values('SchedulerFromDataBase_Sched 28_0',249,'2015-06-30 16:59:23.565','2015-07-01 16:59:23.565','',0,null,'semaphore','172.16.185.21',null,null,0,0) /*!*/; # at 923480328 #150630 16:59:23 server id 3 end_log_pos 923480355 Xid = 7327914927 COMMIT/*!*/; # at 923480355 Pattern: group 1: delete, delete and delete group 2: insert, insert and insert (second insert gets reported due to the first) Again the slave splitted the group into two groups in its binlog. Case 4) Again with context, just to be sure... (only context before, since afterwards other tables are busy) #150630 17:23:26 server id 3 end_log_pos 1031374911 GTID 0-3-533992848 cid=7330164232 /*!100001 SET @@session.gtid_seq_no=533992848*//*!*/; BEGIN /*!*/; # at 1031374911 #150630 17:23:26 server id 3 end_log_pos 1031375063 Query thread_id=29456357 exec_time=0 error_code=0 SET TIMESTAMP=1435677806/*!*/; delete from `queueentry` where ((`OID` = 12695140) and (`OptimisticLockField` = 0)) /*!*/; # at 1031375063 #150630 17:23:26 server id 3 end_log_pos 1031375090 Xid = 7330164230 COMMIT/*!*/; # at 1031375090 #150630 17:23:26 server id 3 end_log_pos 1031375130 GTID 0-3-533992849 cid=7330164232 /*!100001 SET @@session.gtid_seq_no=533992849*//*!*/; BEGIN /*!*/; # at 1031375130 #150630 17:23:26 server id 3 end_log_pos 1031375282 Query thread_id=29278185 exec_time=0 error_code=0 SET TIMESTAMP=1435677806/*!*/; delete from `queueentry` where ((`OID` = 12695138) and (`OptimisticLockField` = 0)) /*!*/; # at 1031375282 #150630 17:23:26 server id 3 end_log_pos 1031375309 Xid = 7330164239 COMMIT/*!*/; # at 1031375309 #150630 17:23:26 server id 3 end_log_pos 1031375349 GTID 0-3-533992850 cid=7330164232 /*!100001 SET @@session.gtid_seq_no=533992850*//*!*/; BEGIN /*!*/; # at 1031375349 #150630 17:23:26 server id 3 end_log_pos 1031375377 Intvar SET INSERT_ID=12695364/*!*/; # at 1031375377 #150630 17:23:26 server id 3 end_log_pos 1031375795 Query thread_id=29456384 exec_time=0 error_code=0 SET TIMESTAMP=1435677806/*!*/; insert into `queueentry`(`identry`,`idqueue`,`entrydate`,`removedate`,`info`,`prio`,`reserved`,`ObjectC ontext`,`OriginHost`,`ReservingHost`,`UnreserveDate`,`ReservationCounter`,`OptimisticLockField`)values( 'SchedulerFromDataBase_Sched 32_0',249,'2015-06-30 17:23:26.396','2015-07-01 17:23:26.396','',0,null,'s emaphore','172.16.185.21',null,null,0,0) /*!*/; # at 1031375795 #150630 17:23:26 server id 3 end_log_pos 1031375822 Xid = 7330164240 COMMIT/*!*/; # at 1031375822 #150630 17:23:26 server id 3 end_log_pos 1031375862 GTID 0-3-533992851 cid=7330164280 /*!100001 SET @@session.gtid_seq_no=533992851*//*!*/; BEGIN /*!*/; # at 1031375862 #150630 17:23:26 server id 3 end_log_pos 1031375890 Intvar SET INSERT_ID=12695365/*!*/; # at 1031375890 #150630 17:23:26 server id 3 end_log_pos 1031376308 Query thread_id=29278186 exec_time=0 error_code=0 SET TIMESTAMP=1435677806/*!*/; insert into `queueentry`(`identry`,`idqueue`,`entrydate`,`removedate`,`info`,`prio`,`reserved`,`ObjectContext`,`OriginHost`,`ReservingHost`,`UnreserveDate`,`ReservationCounter`,`OptimisticLockField`)values('SchedulerFromDataBase_Sched 45_0',249,'2015-06-30 17:23:26.427','2015-07-01 17:23:26.427','',0,null,'semaphore','172.16.185.22',null,null,0,0) /*!*/; # at 1031376308 #150630 17:23:26 server id 3 end_log_pos 1031376335 Xid = 7330164275 COMMIT/*!*/; # at 1031376335 #150630 17:23:26 server id 3 end_log_pos 1031376375 GTID 0-3-533992852 cid=7330164280 /*!100001 SET @@session.gtid_seq_no=533992852*//*!*/; BEGIN /*!*/; # at 1031376375 #150630 17:23:26 server id 3 end_log_pos 1031376403 Intvar SET INSERT_ID=12695366/*!*/; # at 1031376403 #150630 17:23:26 server id 3 end_log_pos 1031376821 Query thread_id=29456384 exec_time=0 error_code=0 SET TIMESTAMP=1435677806/*!*/; insert into `queueentry`(`identry`,`idqueue`,`entrydate`,`removedate`,`info`,`prio`,`reserved`,`ObjectContext`,`OriginHost`,`ReservingHost`,`UnreserveDate`,`ReservationCounter`,`OptimisticLockField`)values('SchedulerFromDataBase_Sched 46_0',249,'2015-06-30 17:23:26.427','2015-07-01 17:23:26.427','',0,null,'semaphore','172.16.185.22',null,null,0,0) /*!*/; # at 1031376821 #150630 17:23:26 server id 3 end_log_pos 1031376848 Xid = 7330164278 COMMIT/*!*/; # at 1031376848 # Pattern: group 1: delete, delete, insert group 2: insert, insert (second insert gets reported due to the first) Again the slave created two groups for group 2 in its own binlog. Case 5) Again with context before, but none after. #150701 0:44:15 server id 3 end_log_pos 747684889 GTID 0-3-535172059 cid=7365441428 /*!100001 SET @@session.gtid_seq_no=535172059*//*!*/; BEGIN /*!*/; # at 747684889 #150701 0:44:15 server id 3 end_log_pos 747684917 Intvar SET INSERT_ID=1191462/*!*/; # at 747684917 #150701 0:44:15 server id 3 end_log_pos 747685235 Query thread_id=30824422 exec_time=0 error_code=0 SET TIMESTAMP=1435704255/*!*/; insert into `abrsd01`(`ABZO05`,`ABZO01`,`ABATID`,`ABSRCE`,`ABBUID`,`ABORID`,`ABOPID`,`ABQRES`,`ABUPUS`, `ABUPBS`,`ABUPDT`,`ABUPTM`,`ABSTAT`,`OptimisticLockField`)values(55298,1,null,'CORD01',0,56578,2107742, 1,'','OPTOSERVICE01',20150701,4414,'NEW',0) /*!*/; # at 747685235 #150701 0:44:15 server id 3 end_log_pos 747685262 Xid = 7365441422 COMMIT/*!*/; # at 747685262 #150701 0:44:15 server id 3 end_log_pos 747685302 GTID 0-3-535172060 cid=7365441428 /*!100001 SET @@session.gtid_seq_no=535172060*//*!*/; BEGIN /*!*/; # at 747685302 #150701 0:44:15 server id 3 end_log_pos 747685746 Query thread_id=29456384 exec_time=0 error_code=0 SET TIMESTAMP=1435704255/*!*/; update `queueentry` set `identry`='057209',`idqueue`=284,`entrydate`='2015-06-30 17:27:46 ',`removedate `='2015-07-01 17:27:46 ',`info`='RB* (Standard)',`prio`=0,`reserved`=null,`ObjectContext`='job',`Origin Host`='172.16.185.22',`ReservingHost`=null,`UnreserveDate`=null,`ReservationCounter`=2,`OptimisticLockF ield`=4 where ((`OID` = 12696865) and (`OptimisticLockField` = 3)) /*!*/; # at 747685746 #150701 0:44:15 server id 3 end_log_pos 747685773 Xid = 7365441437 COMMIT/*!*/; # at 747685773 #150701 0:44:15 server id 3 end_log_pos 747685813 GTID 0-3-535172061 cid=7365441428 /*!100001 SET @@session.gtid_seq_no=535172061*//*!*/; BEGIN /*!*/; # at 747685813 #150701 0:44:15 server id 3 end_log_pos 747685965 Query thread_id=29456385 exec_time=0 error_code=0 SET TIMESTAMP=1435704255/*!*/; delete from `queueentry` where ((`OID` = 12794632) and (`OptimisticLockField` = 0)) /*!*/; # at 747685965 #150701 0:44:15 server id 3 end_log_pos 747685992 Xid = 7365441442 COMMIT/*!*/; # at 747685992 #150701 0:44:15 server id 3 end_log_pos 747686032 GTID 0-3-535172062 cid=7365441467 /*!100001 SET @@session.gtid_seq_no=535172062*//*!*/; BEGIN /*!*/; # at 747686032 #150701 0:44:15 server id 3 end_log_pos 747686060 Intvar SET INSERT_ID=12794638/*!*/; # at 747686060 #150701 0:44:15 server id 3 end_log_pos 747686462 Query thread_id=29456384 exec_time=0 error_code=0 SET TIMESTAMP=1435704255/*!*/; insert into `queueentry`(`identry`,`idqueue`,`entrydate`,`removedate`,`info`,`prio`,`reserved`,`ObjectContext`,`OriginHost`,`ReservingHost`,`UnreserveDate`,`ReservationCounter`,`OptimisticLockField`)values('RemoveFromStock_0',249,'2015-07-01 00:44:14.955','2015-07-01 00:59:14.955','',0,null,'semaphore','172.16.184.9',null,null,0,0) /*!*/; # at 747686462 #150701 0:44:15 server id 3 end_log_pos 747686489 Xid = 7365441465 COMMIT/*!*/; # at 747686489 #150701 0:44:15 server id 3 end_log_pos 747686529 GTID 0-3-535172063 cid=7365441467 /*!100001 SET @@session.gtid_seq_no=535172063*//*!*/; BEGIN /*!*/; # at 747686529 #150701 0:44:15 server id 3 end_log_pos 747686681 Query thread_id=29357465 exec_time=0 error_code=0 SET TIMESTAMP=1435704255/*!*/; delete from `queueentry` where ((`OID` = 12794633) and (`OptimisticLockField` = 0)) /*!*/; # at 747686681 #150701 0:44:15 server id 3 end_log_pos 747686708 Xid = 7365441473 COMMIT/*!*/; # at 747686708 Pattern: group 1: insert, update, delete group 2: insert, delete (the delete gets reported due to the insert) I did not check the slaves binlog here. Now having seen these logs, I must agree, this seems to be a different error than we reported first. It really seems to happen inside a group of transactions (while the other involved two groups of transactions). Another indicator for this is that the new errors happened while the slave was in sync with the master, while the other scenario happened when the slave was behind. Do you need anything more? Waiting for your comments...

          Thomas,

          Thanks for the detailed info, that is really helpful.

          So I should clarify that it is normal (if rare) to get conflicts between
          transactions in parallel replication, the code is supposed to handle it. The
          patch adds printouts for all detected conflicts. The current theory is a
          subtle bug that incorrectly handles a conflict in some specific case.

          So your printouts show 5 cases of deadlocks that were apparently all handled
          correctly.

          So in the 4th case (GTIDs 0-3-533992851 and 0-3-533992852), my potential fix
          triggered:

          insert into `queueentry`(`identry`,`idqueue`,`entrydate`,`removedate`,`info`,`prio`,`reserved`,`ObjectContext`,`OriginHost`,`ReservingHost`,`UnreserveDate`,`ReservationCounter`,`OptimisticLockField`)values('SchedulerFromDataBase_Sched 45_0',249,'2015-06-30 17:23:26.427','2015-07-01 17:23:26.427','',0,null,'semaphore','172.16.185.22',null,null,0,0)
          insert into `queueentry`(`identry`,`idqueue`,`entrydate`,`removedate`,`info`,`prio`,`reserved`,`ObjectContext`,`OriginHost`,`ReservingHost`,`UnreserveDate`,`ReservationCounter`,`OptimisticLockField`)values('SchedulerFromDataBase_Sched 46_0',249,'2015-06-30 17:23:26.427','2015-07-01 17:23:26.427','',0,null,'semaphore','172.16.185.22',null,null,0,0)

          So on the master these two inserts ran without lock conflicts, but on the
          slave the second somehow held a row lock that blocked the first one.

          I am really curious to understand why InnoDB gets a lock conflict here. The
          two inserts are not overlapping on the unique keys. But the second insert
          has (probably) the very next value in the first coloum of a composite unique
          index.

          I have seen one other case of a similar strange conflict, and it was also on
          a composite unique index.

          So now things become very technical, but I am speculating that in some
          particular rare case, maybe InnoDB index split or purge of deleted rows,
          InnoDB needs to take some additional row lock. And this lock is what is
          causing the conflict on the slave (but not on the master).

          Since this lock is not required for transactional consistency, it might be
          that InnoDB releases it early, during or immediately after the insert
          (normally, locks are only released during commit). If so, there is no actual
          deadlock. And it is possible for both inserts to complete and start
          committing, before the kill is detected by the second insert. At this point,
          the following group of transactions can start. If then the second insert
          detects the kill and rolls back, it would be possible for the following
          transactions to see the state of the database before the second insert,
          which is incorrect.

          So in this case we had insert vs. insert, which does not cause any
          problems. But if the same condition happened with an insert vs. a delete, we
          could get the duplicate key bug that you experience.

          And my patch would fix the bug, because it adds an extra check if the second
          transaction was killed, right before starting the following group. So the
          following group would not be started too early.

          So this is all pure speculation. I do not know if InnoDB actually has such
          locking behaviour. But at least it is a somewhat plausible and possible
          explanation.

          So it will be interesting to see if you are able to trigger the bug with my
          patch. Or if you will trigger the "Skip mark_start_commit..." printout with
          an insert+delete.

          There is also a printout "skipping free of GCO..." which, if triggered,
          definitely indicates a bug, so triggering that would also be very
          interesting. This is the condition that caused the slave threads to hang in
          your original tests on earlier MariaDB version.

          So yes, if you are able do let the tests continue, that would be most
          helpful. It is really important to get this bug fixed, but the required
          conditions to trigger it are extremely elusive.

          knielsen Kristian Nielsen added a comment - Thomas, Thanks for the detailed info, that is really helpful. So I should clarify that it is normal (if rare) to get conflicts between transactions in parallel replication, the code is supposed to handle it. The patch adds printouts for all detected conflicts. The current theory is a subtle bug that incorrectly handles a conflict in some specific case. So your printouts show 5 cases of deadlocks that were apparently all handled correctly. So in the 4th case (GTIDs 0-3-533992851 and 0-3-533992852), my potential fix triggered: insert into `queueentry`(`identry`,`idqueue`,`entrydate`,`removedate`,`info`,`prio`,`reserved`,`ObjectContext`,`OriginHost`,`ReservingHost`,`UnreserveDate`,`ReservationCounter`,`OptimisticLockField`)values('SchedulerFromDataBase_Sched 45_0',249,'2015-06-30 17:23:26.427','2015-07-01 17:23:26.427','',0,null,'semaphore','172.16.185.22',null,null,0,0) insert into `queueentry`(`identry`,`idqueue`,`entrydate`,`removedate`,`info`,`prio`,`reserved`,`ObjectContext`,`OriginHost`,`ReservingHost`,`UnreserveDate`,`ReservationCounter`,`OptimisticLockField`)values('SchedulerFromDataBase_Sched 46_0',249,'2015-06-30 17:23:26.427','2015-07-01 17:23:26.427','',0,null,'semaphore','172.16.185.22',null,null,0,0) So on the master these two inserts ran without lock conflicts, but on the slave the second somehow held a row lock that blocked the first one. I am really curious to understand why InnoDB gets a lock conflict here. The two inserts are not overlapping on the unique keys. But the second insert has (probably) the very next value in the first coloum of a composite unique index. I have seen one other case of a similar strange conflict, and it was also on a composite unique index. So now things become very technical, but I am speculating that in some particular rare case, maybe InnoDB index split or purge of deleted rows, InnoDB needs to take some additional row lock. And this lock is what is causing the conflict on the slave (but not on the master). Since this lock is not required for transactional consistency, it might be that InnoDB releases it early, during or immediately after the insert (normally, locks are only released during commit). If so, there is no actual deadlock. And it is possible for both inserts to complete and start committing, before the kill is detected by the second insert. At this point, the following group of transactions can start. If then the second insert detects the kill and rolls back, it would be possible for the following transactions to see the state of the database before the second insert, which is incorrect. So in this case we had insert vs. insert, which does not cause any problems. But if the same condition happened with an insert vs. a delete, we could get the duplicate key bug that you experience. And my patch would fix the bug, because it adds an extra check if the second transaction was killed, right before starting the following group. So the following group would not be started too early. So this is all pure speculation. I do not know if InnoDB actually has such locking behaviour. But at least it is a somewhat plausible and possible explanation. So it will be interesting to see if you are able to trigger the bug with my patch. Or if you will trigger the "Skip mark_start_commit..." printout with an insert+delete. There is also a printout "skipping free of GCO..." which, if triggered, definitely indicates a bug, so triggering that would also be very interesting. This is the condition that caused the slave threads to hang in your original tests on earlier MariaDB version. So yes, if you are able do let the tests continue, that would be most helpful. It is really important to get this bug fixed, but the required conditions to trigger it are extremely elusive.

          Thanks for the explanations, this helps me to prefilter the results we get a bit. It also explains, why the cases we saw so far do not match the patterns we have seen before. So from now on I will check once daily, which new "MDEV8302 messages" we got and report them. Any "Skip mark_start_commit" or "skipping free of GCO" I will add the binlog data for. In all other cases, I will do so on request (we keep the binlog of the master for round about 48 hours). Of course all replication errors will be reported, too.

          We just got another one:

          MDEV8302: Deadlock kill GTID 0-3-536584349 (subid 537187205 in_commit=0) due to blocking GTID 0-3-536584348 (subid 537187204 in_commit=0)
          MDEV8302: Skip mark_start_commit(GTID 0-3-536584349) due to killed
          MDEV8302: Got deadlock kill in GTID 0-3-536584349 (subid 537187205 in_commit=0)
          MDEV8302: Retry #1 of GTID 0-3-536584349

          But again two times insert:

          #150701 11:36:10 server id 3  end_log_pos 641573114     GTID 0-3-536584348 cid=7397729641
          /*!100001 SET @@session.gtid_seq_no=536584348*//*!*/;
          BEGIN
          /*!*/;
          # at 641573114
          #150701 11:36:10 server id 3  end_log_pos 641573142     Intvar
          SET INSERT_ID=12883795/*!*/;
          # at 641573142
          #150701 11:36:10 server id 3  end_log_pos 641573551     Query   thread_id=29456384      exec_ti
          me=0     error_code=0
          SET TIMESTAMP=1435743370/*!*/;
          insert into `queueentry`(`identry`,`idqueue`,`entrydate`,`removedate`,`info`,`prio`,`reserved`,
          `ObjectContext`,`OriginHost`,`ReservingHost`,`UnreserveDate`,`ReservationCounter`,`OptimisticLo
          ckField`)values('SemaphoreWebservice1_0',249,'2015-07-01 11:36:10.463','2015-07-01 12:36:10.463
          ','',0,null,'semaphore','172.16.185.107',null,null,0,0)
          /*!*/;
          # at 641573551
          #150701 11:36:10 server id 3  end_log_pos 641573578     Xid = 7397729639
          COMMIT/*!*/;
          # at 641573578
          #150701 11:36:10 server id 3  end_log_pos 641573618     GTID 0-3-536584349 cid=7397729641
          /*!100001 SET @@session.gtid_seq_no=536584349*//*!*/;
          BEGIN
          /*!*/;
          # at 641573618
          #150701 11:36:10 server id 3  end_log_pos 641573646     Intvar
          SET INSERT_ID=12883796/*!*/;
          # at 641573646
          #150701 11:36:10 server id 3  end_log_pos 641574055     Query   thread_id=29278186      exec_time=0     error_code=0
          SET TIMESTAMP=1435743370/*!*/;
          insert into `queueentry`(`identry`,`idqueue`,`entrydate`,`removedate`,`info`,`prio`,`reserved`,`ObjectContext`,`OriginHost`,`ReservingHost`,`UnreserveDate`,`ReservationCounter`,`OptimisticLockField`)values('SemaphoreWebservice2_0',249,'2015-07-01 11:36:10.479','2015-07-01 12:36:10.479','',0,null,'semaphore','172.16.185.115',null,null,0,0)
          /*!*/;
          # at 641574055
          #150701 11:36:10 server id 3  end_log_pos 641574082     Xid = 7397729648
          COMMIT/*!*/;
          # at 641574082

          thomas.mischke Thomas Mischke added a comment - Thanks for the explanations, this helps me to prefilter the results we get a bit. It also explains, why the cases we saw so far do not match the patterns we have seen before. So from now on I will check once daily, which new "MDEV8302 messages" we got and report them. Any "Skip mark_start_commit" or "skipping free of GCO" I will add the binlog data for. In all other cases, I will do so on request (we keep the binlog of the master for round about 48 hours). Of course all replication errors will be reported, too. We just got another one: MDEV8302: Deadlock kill GTID 0-3-536584349 (subid 537187205 in_commit=0) due to blocking GTID 0-3-536584348 (subid 537187204 in_commit=0) MDEV8302: Skip mark_start_commit(GTID 0-3-536584349) due to killed MDEV8302: Got deadlock kill in GTID 0-3-536584349 (subid 537187205 in_commit=0) MDEV8302: Retry #1 of GTID 0-3-536584349 But again two times insert: #150701 11:36:10 server id 3 end_log_pos 641573114 GTID 0-3-536584348 cid=7397729641 /*!100001 SET @@session.gtid_seq_no=536584348*//*!*/; BEGIN /*!*/; # at 641573114 #150701 11:36:10 server id 3 end_log_pos 641573142 Intvar SET INSERT_ID=12883795/*!*/; # at 641573142 #150701 11:36:10 server id 3 end_log_pos 641573551 Query thread_id=29456384 exec_ti me=0 error_code=0 SET TIMESTAMP=1435743370/*!*/; insert into `queueentry`(`identry`,`idqueue`,`entrydate`,`removedate`,`info`,`prio`,`reserved`, `ObjectContext`,`OriginHost`,`ReservingHost`,`UnreserveDate`,`ReservationCounter`,`OptimisticLo ckField`)values('SemaphoreWebservice1_0',249,'2015-07-01 11:36:10.463','2015-07-01 12:36:10.463 ','',0,null,'semaphore','172.16.185.107',null,null,0,0) /*!*/; # at 641573551 #150701 11:36:10 server id 3 end_log_pos 641573578 Xid = 7397729639 COMMIT/*!*/; # at 641573578 #150701 11:36:10 server id 3 end_log_pos 641573618 GTID 0-3-536584349 cid=7397729641 /*!100001 SET @@session.gtid_seq_no=536584349*//*!*/; BEGIN /*!*/; # at 641573618 #150701 11:36:10 server id 3 end_log_pos 641573646 Intvar SET INSERT_ID=12883796/*!*/; # at 641573646 #150701 11:36:10 server id 3 end_log_pos 641574055 Query thread_id=29278186 exec_time=0 error_code=0 SET TIMESTAMP=1435743370/*!*/; insert into `queueentry`(`identry`,`idqueue`,`entrydate`,`removedate`,`info`,`prio`,`reserved`,`ObjectContext`,`OriginHost`,`ReservingHost`,`UnreserveDate`,`ReservationCounter`,`OptimisticLockField`)values('SemaphoreWebservice2_0',249,'2015-07-01 11:36:10.479','2015-07-01 12:36:10.479','',0,null,'semaphore','172.16.185.115',null,null,0,0) /*!*/; # at 641574055 #150701 11:36:10 server id 3 end_log_pos 641574082 Xid = 7397729648 COMMIT/*!*/; # at 641574082

          Now we have a "skip mark_start_commit" with an "insert and delete" combination:

          MDEV8302: Deadlock kill GTID 0-3-536723145 (subid 537326001 in_commit=0) due to blocking GTID 0-3-536723144 (subid 537326000 in_commit=0)
          MDEV8302: Skip mark_start_commit(GTID 0-3-536723145) due to killed
          MDEV8302: Got deadlock kill in GTID 0-3-536723145 (subid 537326001 in_commit=0)
          MDEV8302: Retry #1 of GTID 0-3-536723145

          Binlog:

          #150701 12:17:14 server id 3  end_log_pos 824521702     GTID 0-3-536723144 cid=7400114100
          /*!100001 SET @@session.gtid_seq_no=536723144*//*!*/;
          BEGIN
          /*!*/;
          # at 824521702
          #150701 12:17:14 server id 3  end_log_pos 824521854     Query   thread_id=29456357      exec_time=0     error_code=0
          SET TIMESTAMP=1435745834/*!*/;
          delete from `queueentry` where ((`OID` = 12896169) and (`OptimisticLockField` = 0))
          /*!*/;
          # at 824521854
          #150701 12:17:14 server id 3  end_log_pos 824521881     Xid = 7400114097
          COMMIT/*!*/;
          # at 824521881
          #150701 12:17:14 server id 3  end_log_pos 824521921     GTID 0-3-536723145 cid=7400114100
          /*!100001 SET @@session.gtid_seq_no=536723145*//*!*/;
          BEGIN
          /*!*/;
          # at 824521921
          #150701 12:17:14 server id 3  end_log_pos 824521949     Intvar
          SET INSERT_ID=12896176/*!*/;
          # at 824521949
          #150701 12:17:14 server id 3  end_log_pos 824522358     Query   thread_id=29456734      exec_time=0     error_code=0
          SET TIMESTAMP=1435745834/*!*/;
          insert into `queueentry`(`identry`,`idqueue`,`entrydate`,`removedate`,`info`,`prio`,`reserved`,
          `ObjectContext`,`OriginHost`,`ReservingHost`,`UnreserveDate`,`ReservationCounter`,`OptimisticLo
          ckField`)values('SemaphoreWebservice1_0',249,'2015-07-01 12:17:14.354','2015-07-01 13:17:14.354
          ','',0,null,'semaphore','172.16.185.109',null,null,0,0)
          /*!*/;
          # at 824522358
          #150701 12:17:14 server id 3  end_log_pos 824522385     Xid = 7400114101
          COMMIT/*!*/;
          # at 824522385

          The complete log since my last message looks like this:

          MDEV8302: Deadlock kill GTID 0-3-536723145 (subid 537326001 in_commit=0) due to blocking GTID 0-3-536723144 (subid 537326000 in_commit=0)
          MDEV8302: Skip mark_start_commit(GTID 0-3-536723145) due to killed
          MDEV8302: Got deadlock kill in GTID 0-3-536723145 (subid 537326001 in_commit=0)
          MDEV8302: Retry #1 of GTID 0-3-536723145
           
          MDEV8302: Deadlock kill GTID 0-3-536735970 (subid 537338826 in_commit=0) due to blocking GTID 0-3-536735969 (subid 537338825 in_commit=0)
          MDEV8302: Got deadlock kill in GTID 0-3-536735970 (subid 537338826 in_commit=0)
          MDEV8302: Retry #1 of GTID 0-3-536735970
           
          MDEV8302: Deadlock kill GTID 0-3-536968943 (subid 537571799 in_commit=0) due to blocking GTID 0-3-536968941 (subid 537571797 in_commit=0)
          MDEV8302: Skip mark_start_commit(GTID 0-3-536968943) due to killed
          MDEV8302: Got deadlock kill in GTID 0-3-536968943 (subid 537571799 in_commit=0)
          MDEV8302: Retry #1 of GTID 0-3-536968943

          The last case is an "insert, insert" combination together with a transaction from another table, therefore the GTID differs by more than one.

          In case of any interest in further details...

          thomas.mischke Thomas Mischke added a comment - Now we have a "skip mark_start_commit" with an "insert and delete" combination: MDEV8302: Deadlock kill GTID 0-3-536723145 (subid 537326001 in_commit=0) due to blocking GTID 0-3-536723144 (subid 537326000 in_commit=0) MDEV8302: Skip mark_start_commit(GTID 0-3-536723145) due to killed MDEV8302: Got deadlock kill in GTID 0-3-536723145 (subid 537326001 in_commit=0) MDEV8302: Retry #1 of GTID 0-3-536723145 Binlog: #150701 12:17:14 server id 3 end_log_pos 824521702 GTID 0-3-536723144 cid=7400114100 /*!100001 SET @@session.gtid_seq_no=536723144*//*!*/; BEGIN /*!*/; # at 824521702 #150701 12:17:14 server id 3 end_log_pos 824521854 Query thread_id=29456357 exec_time=0 error_code=0 SET TIMESTAMP=1435745834/*!*/; delete from `queueentry` where ((`OID` = 12896169) and (`OptimisticLockField` = 0)) /*!*/; # at 824521854 #150701 12:17:14 server id 3 end_log_pos 824521881 Xid = 7400114097 COMMIT/*!*/; # at 824521881 #150701 12:17:14 server id 3 end_log_pos 824521921 GTID 0-3-536723145 cid=7400114100 /*!100001 SET @@session.gtid_seq_no=536723145*//*!*/; BEGIN /*!*/; # at 824521921 #150701 12:17:14 server id 3 end_log_pos 824521949 Intvar SET INSERT_ID=12896176/*!*/; # at 824521949 #150701 12:17:14 server id 3 end_log_pos 824522358 Query thread_id=29456734 exec_time=0 error_code=0 SET TIMESTAMP=1435745834/*!*/; insert into `queueentry`(`identry`,`idqueue`,`entrydate`,`removedate`,`info`,`prio`,`reserved`, `ObjectContext`,`OriginHost`,`ReservingHost`,`UnreserveDate`,`ReservationCounter`,`OptimisticLo ckField`)values('SemaphoreWebservice1_0',249,'2015-07-01 12:17:14.354','2015-07-01 13:17:14.354 ','',0,null,'semaphore','172.16.185.109',null,null,0,0) /*!*/; # at 824522358 #150701 12:17:14 server id 3 end_log_pos 824522385 Xid = 7400114101 COMMIT/*!*/; # at 824522385 The complete log since my last message looks like this: MDEV8302: Deadlock kill GTID 0-3-536723145 (subid 537326001 in_commit=0) due to blocking GTID 0-3-536723144 (subid 537326000 in_commit=0) MDEV8302: Skip mark_start_commit(GTID 0-3-536723145) due to killed MDEV8302: Got deadlock kill in GTID 0-3-536723145 (subid 537326001 in_commit=0) MDEV8302: Retry #1 of GTID 0-3-536723145   MDEV8302: Deadlock kill GTID 0-3-536735970 (subid 537338826 in_commit=0) due to blocking GTID 0-3-536735969 (subid 537338825 in_commit=0) MDEV8302: Got deadlock kill in GTID 0-3-536735970 (subid 537338826 in_commit=0) MDEV8302: Retry #1 of GTID 0-3-536735970   MDEV8302: Deadlock kill GTID 0-3-536968943 (subid 537571799 in_commit=0) due to blocking GTID 0-3-536968941 (subid 537571797 in_commit=0) MDEV8302: Skip mark_start_commit(GTID 0-3-536968943) due to killed MDEV8302: Got deadlock kill in GTID 0-3-536968943 (subid 537571799 in_commit=0) MDEV8302: Retry #1 of GTID 0-3-536968943 The last case is an "insert, insert" combination together with a transaction from another table, therefore the GTID differs by more than one. In case of any interest in further details...

          Thanks. So in this case, it is the insert that is killed and retried because
          it blocks the delete.

          I guess before the printouts we only saw the case of insert+delete where the
          delete was killed. Because in order to get the duplicate key bug, we need a
          delete to be killed, and a following insert with the same key that then
          runs too early, before the killed delete has been committed.

          I suppose we will have to wait and see if we can trigger the bug with the
          patch. If not, we can try running with the printouts but without my
          speculative fix - and see if we can reproduce without it.

          But I guess the original problem is sufficiently rare that it might be hard
          to say conclusively whether the problem is gone, or just did not show
          itself...

          Meanwhile, based on your detailed information, I will try to set up some
          tests and see if I can reproduce this conflict locally...

          knielsen Kristian Nielsen added a comment - Thanks. So in this case, it is the insert that is killed and retried because it blocks the delete. I guess before the printouts we only saw the case of insert+delete where the delete was killed. Because in order to get the duplicate key bug, we need a delete to be killed, and a following insert with the same key that then runs too early, before the killed delete has been committed. I suppose we will have to wait and see if we can trigger the bug with the patch. If not, we can try running with the printouts but without my speculative fix - and see if we can reproduce without it. But I guess the original problem is sufficiently rare that it might be hard to say conclusively whether the problem is gone, or just did not show itself... Meanwhile, based on your detailed information, I will try to set up some tests and see if I can reproduce this conflict locally...

          No further luck so far. Never seen that the "delete" got killed.

          thomas.mischke Thomas Mischke added a comment - No further luck so far. Never seen that the "delete" got killed.

          Again 10 cases, where the insert gets killed, not the delete. We keep watching.

          thomas.mischke Thomas Mischke added a comment - Again 10 cases, where the insert gets killed, not the delete. We keep watching.

          I am convinced now that in the cases I take a closer look at, always the insert gets killed, never the delete. And before we did the update, we normally needed less that 2 days to get the replication error. Therefore we would like to test your patch without the actual patch code, just with the log messages, just to see, if the error is still happening, or if for whatever reason the error went away. Can you provide us such a patch or tell us, which flag to set during compiling?

          thomas.mischke Thomas Mischke added a comment - I am convinced now that in the cases I take a closer look at, always the insert gets killed, never the delete. And before we did the update, we normally needed less that 2 days to get the replication error. Therefore we would like to test your patch without the actual patch code, just with the log messages, just to see, if the error is still happening, or if for whatever reason the error went away. Can you provide us such a patch or tell us, which flag to set during compiling?

          Here you are. This is an incremental patch on top of the first patch:

          http://lists.askmonty.org/pipermail/commits/2015-July/008148.html

          Or alternatively, this is a full patch on top of vanilla 10.0:

          http://lists.askmonty.org/pipermail/commits/2015-July/008149.html

          (The end result is the same, just a matter of which tree you prefer to start
          with). With this new patch, there are no change in what the code does, just
          added printouts. There is a printout "MDEV8302: mark_start_commit(GTID
          X-X-X) after killed" in the place where the patch would have triggered
          before, but no change in the behaviour, so getting this printout in
          connection with a failure would be very interesting, of course.

          But we will see. I do not think the first patch would affect the possibility
          of seeing the delete killed - just possibly avoid the duplicate key
          violation in this case. But this is a race bug, and any change can affect
          the likeliness of triggering it.

          So we will see.

          knielsen Kristian Nielsen added a comment - Here you are. This is an incremental patch on top of the first patch: http://lists.askmonty.org/pipermail/commits/2015-July/008148.html Or alternatively, this is a full patch on top of vanilla 10.0: http://lists.askmonty.org/pipermail/commits/2015-July/008149.html (The end result is the same, just a matter of which tree you prefer to start with). With this new patch, there are no change in what the code does, just added printouts. There is a printout "MDEV8302: mark_start_commit(GTID X-X-X) after killed" in the place where the patch would have triggered before, but no change in the behaviour, so getting this printout in connection with a failure would be very interesting, of course. But we will see. I do not think the first patch would affect the possibility of seeing the delete killed - just possibly avoid the duplicate key violation in this case. But this is a race bug, and any change can affect the likeliness of triggering it. So we will see.

          Sorry that it took so long to answer, I was sick a few days... But today I was able to reproduce the "duplicate key" errors, some without mark_start_commit involved, and one with mark_start_commit. We were using the version with just the logging, so the replication stopped in all cases, and using "slave start" the slave started again without further actions. Here are the details:

          MDEV8302: Deadlock kill GTID 0-3-589790787 (subid 590503582 in_commit=0) due to blocking GTID 0-3-589790786 (subid 590503581 in_commit=0)
          MDEV8302: Got deadlock kill in GTID 0-3-589790787 (subid 590503582 in_commit=1)
          MDEV8302: Retry #1 of GTID 0-3-589790787
          150720 13:43:31 [ERROR] Slave SQL: Error 'Duplicate entry 'semtest123456789_0-249' for key 'identry_idqueue'' on query. Default database: 'optovision'. Query: 'insert into `queueentry`(`identry`,`idqueue`,`entrydate`,`removedate`,`info`,`prio`,`reserved`,`ObjectContext`,`OriginHost`,`ReservingHost`,`UnreserveDate`,`ReservationCounter`,`OptimisticLockField`)values('semtest123456789_0',249,'2015-07-20 13:43:23.477','2015-07-20 13:48:23.475','',0,null,'semaphore','',null,null,0,0)', Gtid 0-3-589790789, Internal MariaDB error code: 1062
          150720 13:43:31 [Warning] Slave: Duplicate entry 'semtest123456789_0-249' for key 'identry_idqueue' Error_code: 1062
          150720 13:43:31 [ERROR] Error running query, slave SQL thread aborted. Fix the problem, and restart the slave SQL thread with "SLAVE START". We stopped at log 'mysql-bin.007960' position 154941507; GTID position '77-3-552,0-3-589790785,3-3-2,2-3-690710,17-3-180,1-3-7'
          150720 13:43:31 [Note] Slave SQL thread exiting, replication stopped in log 'mysql-bin.007960' at position 154942426; GTID position '77-3-552,0-3-589790788,3-3-2,2-3-690710,17-3-180,1-3-7'

          The binlog looks as follows:

          #150720 13:43:24 server id 3  end_log_pos 154941547     GTID 0-3-589790786 cid=8345984276
          /*!100001 SET @@session.gtid_seq_no=589790786*//*!*/;
          BEGIN
          /*!*/;
          # at 154941547
          #150720 13:43:24 server id 3  end_log_pos 154941575     Intvar
          SET INSERT_ID=17489759/*!*/;
          # at 154941575
          #150720 13:43:24 server id 3  end_log_pos 154941961     Query   thread_id=34916277      exec_time=0     error_code=0
          SET TIMESTAMP=1437392604/*!*/;
          insert into `queueentry`(`identry`,`idqueue`,`entrydate`,`removedate`,`info`,`prio`,`reserved`,`ObjectContext`,`OriginHost`,`ReservingHost`,`UnreserveDate`,`ReservationCounter`,`OptimisticLockField`)values('
          semtest1233_0',249,'2015-07-20 13:43:23.462','2015-07-20 13:48:23.460','',0,null,'semaphore','',null,null,0,0)
          /*!*/;
          # at 154941961
          #150720 13:43:24 server id 3  end_log_pos 154941988     Xid = 8345984273
          COMMIT/*!*/;
          # at 154941988
          #150720 13:43:24 server id 3  end_log_pos 154942028     GTID 0-3-589790787 cid=8345984276
          /*!100001 SET @@session.gtid_seq_no=589790787*//*!*/;
          BEGIN
          /*!*/;
          # at 154942028
          #150720 13:43:24 server id 3  end_log_pos 154942180     Query   thread_id=34915532      exec_time=0     error_code=0
          SET TIMESTAMP=1437392604/*!*/;
          delete from `queueentry` where ((`OID` = 17489757) and (`OptimisticLockField` = 0))
          /*!*/;
          # at 154942180
          #150720 13:43:24 server id 3  end_log_pos 154942207     Xid = 8345984280
          COMMIT/*!*/;
          # at 154942207
          #150720 13:43:24 server id 3  end_log_pos 154942247     GTID 0-3-589790788 cid=8345984320
          /*!100001 SET @@session.gtid_seq_no=589790788*//*!*/;
          BEGIN
          /*!*/;
          # at 154942247
          #150720 13:43:24 server id 3  end_log_pos 154942399     Query   thread_id=34916277      exec_time=0     error_code=0
          SET TIMESTAMP=1437392604/*!*/;
          delete from `queueentry` where ((`OID` = 17489759) and (`OptimisticLockField` = 0))
          /*!*/;
          # at 154942399
          #150720 13:43:24 server id 3  end_log_pos 154942426     Xid = 8345984315
          COMMIT/*!*/;
          # at 154942426
          #150720 13:43:24 server id 3  end_log_pos 154942466     GTID 0-3-589790789 cid=8345984320
          /*!100001 SET @@session.gtid_seq_no=589790789*//*!*/;
          BEGIN
          /*!*/;
          # at 154942466
          #150720 13:43:24 server id 3  end_log_pos 154942494     Intvar
          SET INSERT_ID=17489761/*!*/;
          # at 154942494
          #150720 13:43:24 server id 3  end_log_pos 154942885     Query   thread_id=34915532      exec_time=0     error_code=0
          SET TIMESTAMP=1437392604/*!*/;
          insert into `queueentry`(`identry`,`idqueue`,`entrydate`,`removedate`,`info`,`prio`,`reserved`,`ObjectContext`,`OriginHost`,`ReservingHost`,`UnreserveDate`,`ReservationCounter`,`OptimisticLockField`)values('semtest123456789_0',249,'2015-07-20 13:43:23.477','2015-07-20 13:48:23.475','',0,null,'semaphore','',null,null,0,0)
          /*!*/;
          # at 154942885
          #150720 13:43:24 server id 3  end_log_pos 154942912     Xid = 8345984316
          COMMIT/*!*/;
          # at 154942912

          So in the first group we have:

          • insert (some value)
          • delete OID 17489757
            In the second group we have:
          • delete OID 17489759
          • insert semtest123456789_0-249

          A little bit back in the binlog I found that OID 17489757 corresponds to semtest123456789_0-249:

          #150720 13:43:24 server id 3  end_log_pos 154940842     GTID 0-3-589790784 cid=8345984237
          /*!100001 SET @@session.gtid_seq_no=589790784*//*!*/;
          BEGIN
          /*!*/;
          # at 154940842
          #150720 13:43:24 server id 3  end_log_pos 154940870     Intvar
          SET INSERT_ID=17489757/*!*/;
          # at 154940870
          #150720 13:43:24 server id 3  end_log_pos 154941261     Query   thread_id=34915532      exec_time=0     error_code=0
          SET TIMESTAMP=1437392604/*!*/;
          insert into `queueentry`(`identry`,`idqueue`,`entrydate`,`removedate`,`info`,`prio`,`reserved`,`ObjectContext`,`OriginHost`,`ReservingHost`,`UnreserveDate`,`ReservationCounter`,`OptimisticLockField`)values('semtest123456789_0',249,'2015-07-20 13:43:23.449','2015-07-20 13:48:23.448','',0,null,'semaphore','',null,null,0,0)
          /*!*/;
          # at 154941261

          So the delete from group 1 gets killed due to the deadlock , and while the retry is in progress the insert from the second group starts to run and gets the duplicate key error.

          After starting the slave again it only took seconds to get the next error:

          MDEV8302: Deadlock kill GTID 0-3-589790816 (subid 590504252 in_commit=0) due to blocking GTID 0-3-589790815 (subid 590504251 in_commit=0)
          MDEV8302: Got deadlock kill in GTID 0-3-589790816 (subid 590504252 in_commit=1)
          MDEV8302: Retry #1 of GTID 0-3-589790816
          150720 13:58:03 [ERROR] Slave SQL: Error 'Duplicate entry 'semtest123456789_0-249' for key 'identry_idqueue'' on query. Default database: 'optovision'. Query: 'insert into `queueentry`(`identry`,`idqueue`,`entrydate`,`removedate`,`info`,`prio`,`reserved`,`ObjectContext`,`OriginHost`,`ReservingHost`,`UnreserveDate`,`ReservationCounter`,`OptimisticLockField`)values('semtest123456789_0',249,'2015-07-20 13:43:23.674','2015-07-20 13:48:23.673','',0,null,'semaphore','',null,null,0,0)', Gtid 0-3-589790818, Internal MariaDB error code: 1062
          150720 13:58:03 [Warning] Slave: Duplicate entry 'semtest123456789_0-249' for key 'identry_idqueue' Error_code: 1062
          MDEV8302: Deadlock kill GTID 0-3-589790818 (subid 590504254 in_commit=0) due to blocking GTID 0-3-589790816 (subid 590504252 in_commit=0)
          150720 13:58:03 [ERROR] Error running query, slave SQL thread aborted. Fix the problem, and restart the slave SQL thread with "SLAVE START". We stopped at log 'mysql-bin.007960' position 154952396; GTID position '77-3-552,0-3-589790815,3-3-2,2-3-690710,17-3-180,1-3-7'
          150720 13:58:03 [Note] Slave SQL thread exiting, replication stopped in log 'mysql-bin.007960' at position 154952834; GTID position '77-3-552,0-3-589790817,3-3-2,2-3-690710,17-3-180,1-3-7'

          In this case the log messages are in a different order. The binlog looks as follows:

          #150720 13:43:24 server id 3  end_log_pos 154951955     GTID 0-3-589790815 cid=8345984957
          /*!100001 SET @@session.gtid_seq_no=589790815*//*!*/;
          BEGIN
          /*!*/;
          # at 154951955
          #150720 13:43:24 server id 3  end_log_pos 154951983     Intvar
          SET INSERT_ID=17489795/*!*/;
          # at 154951983
          #150720 13:43:24 server id 3  end_log_pos 154952369     Query   thread_id=34916277      exec_time=0     error_code=0
          SET TIMESTAMP=1437392604/*!*/;
          insert into `queueentry`(`identry`,`idqueue`,`entrydate`,`removedate`,`info`,`prio`,`reserved`,`ObjectContext`,`OriginHost`,`ReservingHost`,`UnreserveDate`,`ReservationCounter`,`OptimisticLockField`)values('
          semtest1233_0',249,'2015-07-20 13:43:23.660','2015-07-20 13:48:23.659','',0,null,'semaphore','',null,null,0,0)
          /*!*/;
          # at 154952369
          #150720 13:43:24 server id 3  end_log_pos 154952396     Xid = 8345984954
          COMMIT/*!*/;
          # at 154952396
          #150720 13:43:24 server id 3  end_log_pos 154952436     GTID 0-3-589790816 cid=8345984957
          /*!100001 SET @@session.gtid_seq_no=589790816*//*!*/;
          BEGIN
          /*!*/;
          # at 154952436
          #150720 13:43:24 server id 3  end_log_pos 154952588     Query   thread_id=34915532      exec_time=0     error_code=0
          SET TIMESTAMP=1437392604/*!*/;
          delete from `queueentry` where ((`OID` = 17489792) and (`OptimisticLockField` = 0))
          /*!*/;
          # at 154952588
          #150720 13:43:24 server id 3  end_log_pos 154952615     Xid = 8345984965
          COMMIT/*!*/;
          # at 154952615
          #150720 13:43:24 server id 3  end_log_pos 154952655     GTID 0-3-589790817 cid=8345985010
          /*!100001 SET @@session.gtid_seq_no=589790817*//*!*/;
          BEGIN
          /*!*/;
          # at 154952655
          #150720 13:43:24 server id 3  end_log_pos 154952807     Query   thread_id=34916277      exec_time=0     error_code=0
          SET TIMESTAMP=1437392604/*!*/;
          delete from `queueentry` where ((`OID` = 17489795) and (`OptimisticLockField` = 0))
          /*!*/;
          # at 154952807
          #150720 13:43:24 server id 3  end_log_pos 154952834     Xid = 8345985005
          COMMIT/*!*/;
          # at 154952834
          #150720 13:43:24 server id 3  end_log_pos 154952874     GTID 0-3-589790818 cid=8345985010
          /*!100001 SET @@session.gtid_seq_no=589790818*//*!*/;
          BEGIN
          /*!*/;
          # at 154952874
          #150720 13:43:24 server id 3  end_log_pos 154952902     Intvar
          SET INSERT_ID=17489797/*!*/;
          # at 154952902
          #150720 13:43:24 server id 3  end_log_pos 154953293     Query   thread_id=34915524      exec_time=0     error_code=0
          SET TIMESTAMP=1437392604/*!*/;
          insert into `queueentry`(`identry`,`idqueue`,`entrydate`,`removedate`,`info`,`prio`,`reserved`,`ObjectContext`,`OriginHost`,`ReservingHost`,`UnreserveDate`,`ReservationCounter`,`OptimisticLockField`)values('semtest123456789_0',249,'2015-07-20 13:43:23.674','2015-07-20 13:48:23.673','',0,null,'semaphore','',null,null,0,0)
          /*!*/;
          # at 154953293
          #150720 13:43:24 server id 3  end_log_pos 154953320     Xid = 8345985004
          COMMIT/*!*/;
          # at 154953320

          Group 1 has two statements:

          • insert (some value)
          • delete OID 17489792
            Group 2 has:
          • delete
          • insert semtest123456789_0-249

          Again the OID 17489792 matches the key value of the insert (semtest123456789_0-249).

          Again I restarted the slave, and the next error looks as follows (again different logging):

          MDEV8302: Deadlock kill GTID 0-3-589791379 (subid 590505427 in_commit=0) due to blocking GTID 0-3-589791378 (subid 590505426 in_commit=0)
          MDEV8302: Got deadlock kill in GTID 0-3-589791379 (subid 590505427 in_commit=1)
          MDEV8302: Retry #1 of GTID 0-3-589791379
          150720 14:09:48 [ERROR] Slave SQL: Error 'Duplicate entry 'semtest123456789_0-249' for key 'identry_idqueue'' on query. Default database: 'optovision'. Query: 'insert into `queueentry`(`identry`,`idqueue`,`entrydate`,`removedate`,`info`,`prio`,`reserved`,`ObjectContext`,`OriginHost`,`ReservingHost`,`UnreserveDate`,`ReservationCounter`,`OptimisticLockField`)values('semtest123456789_0',249,'2015-07-20 13:43:26.960','2015-07-20 13:48:26.958','',0,null,'semaphore','',null,null,0,0)', Gtid 0-3-589791380, Internal MariaDB error code: 1062
          150720 14:09:48 [Warning] Slave: Duplicate entry 'semtest123456789_0-249' for key 'identry_idqueue' Error_code: 1062
          150720 14:09:48 [ERROR] Error running query, slave SQL thread aborted. Fix the problem, and restart the slave SQL thread with "SLAVE START". We stopped at log 'mysql-bin.007960' position 155182289; GTID position '77-3-552,0-3-589791378,3-3-2,2-3-690710,17-3-180,1-3-7'
          150720 14:09:48 [ERROR] Slave (additional info): Commit failed due to failure of an earlier commit on which this one depends Error_code: 1964
          150720 14:09:48 [Warning] Slave: Commit failed due to failure of an earlier commit on which this one depends Error_code: 1964
          150720 14:09:48 [ERROR] Error running query, slave SQL thread aborted. Fix the problem, and restart the slave SQL thread with "SLAVE START". We stopped at log 'mysql-bin.007960' position 155182508; GTID position '77-3-552,0-3-589791379,3-3-2,2-3-690710,17-3-180,1-3-7'
          150720 14:09:48 [ERROR] Slave (additional info): Commit failed due to failure of an earlier commit on which this one depends Error_code: 1964
          150720 14:09:48 [Warning] Slave: Commit failed due to failure of an earlier commit on which this one depends Error_code: 1964
          150720 14:09:48 [ERROR] Error running query, slave SQL thread aborted. Fix the problem, and restart the slave SQL thread with "SLAVE START". We stopped at log 'mysql-bin.007960' position 155182508; GTID position '77-3-552,0-3-589791379,3-3-2,2-3-690710,17-3-180,1-3-7'
          150720 14:09:48 [ERROR] Slave (additional info): Commit failed due to failure of an earlier commit on which this one depends Error_code: 1964
          150720 14:09:48 [Warning] Slave: Commit failed due to failure of an earlier commit on which this one depends Error_code: 1964
          150720 14:09:48 [ERROR] Error running query, slave SQL thread aborted. Fix the problem, and restart the slave SQL thread with "SLAVE START". We stopped at log 'mysql-bin.007960' position 155182508; GTID position '77-3-552,0-3-589791379,3-3-2,2-3-690710,17-3-180,1-3-7'
          150720 14:09:48 [Note] Slave SQL thread exiting, replication stopped in log 'mysql-bin.007960' at position 155182508; GTID position '77-3-552,0-3-589791379,3-3-2,2-3-690710,17-3-180,1-3-7'
          150720 14:10:02 [ERROR] Error reading packet from server: Lost connection to MySQL server during query ( server_errno=2013)
          150720 14:10:02 [Note] Slave I/O thread killed while reading event
          150720 14:10:02 [Note] Slave I/O thread exiting, read up to log 'mysql-bin.007960', position 263021432; GTID position 77-3-552,0-3-589886520,3-3-2,2-3-690710,1-3-7,17-3-180

          Again the binlog:

          #150720 13:43:27 server id 3  end_log_pos 155181848     GTID 0-3-589791378 cid=8345994929
          /*!100001 SET @@session.gtid_seq_no=589791378*//*!*/;
          BEGIN
          /*!*/;
          # at 155181848
          #150720 13:43:27 server id 3  end_log_pos 155181876     Intvar
          SET INSERT_ID=17490362/*!*/;
          # at 155181876
          #150720 13:43:27 server id 3  end_log_pos 155182262     Query   thread_id=34916277      exec_time=0     error_code=0
          SET TIMESTAMP=1437392607/*!*/;
          insert into `queueentry`(`identry`,`idqueue`,`entrydate`,`removedate`,`info`,`prio`,`reserved`,`ObjectContext`,`OriginHost`,`ReservingHost`,`UnreserveDate`,`ReservationCounter`,`OptimisticLockField`)values('
          semtest1233_0',249,'2015-07-20 13:43:26.946','2015-07-20 13:48:26.945','',0,null,'semaphore','',null,null,0,0)
          /*!*/;
          # at 155182262
          #150720 13:43:27 server id 3  end_log_pos 155182289     Xid = 8345994927
          COMMIT/*!*/;
          # at 155182289
          #150720 13:43:27 server id 3  end_log_pos 155182329     GTID 0-3-589791379 cid=8345994929
          /*!100001 SET @@session.gtid_seq_no=589791379*//*!*/;
          BEGIN
          /*!*/;
          # at 155182329
          #150720 13:43:27 server id 3  end_log_pos 155182481     Query   thread_id=34915525      exec_time=0     error_code=0
          SET TIMESTAMP=1437392607/*!*/;
          delete from `queueentry` where ((`OID` = 17490359) and (`OptimisticLockField` = 0))
          /*!*/;
          # at 155182481
          #150720 13:43:27 server id 3  end_log_pos 155182508     Xid = 8345994933
          COMMIT/*!*/;
          # at 155182508
          #150720 13:43:27 server id 3  end_log_pos 155182548     GTID 0-3-589791380 cid=8345994964
          /*!100001 SET @@session.gtid_seq_no=589791380*//*!*/;
          BEGIN
          /*!*/;
          # at 155182548
          #150720 13:43:27 server id 3  end_log_pos 155182576     Intvar
          SET INSERT_ID=17490364/*!*/;
          # at 155182576
          #150720 13:43:27 server id 3  end_log_pos 155182967     Query   thread_id=34915524      exec_time=0     error_code=0
          SET TIMESTAMP=1437392607/*!*/;
          insert into `queueentry`(`identry`,`idqueue`,`entrydate`,`removedate`,`info`,`prio`,`reserved`,`ObjectContext`,`OriginHost`,`ReservingHost`,`UnreserveDate`,`ReservationCounter`,`OptimisticLockField`)values('semtest123456789_0',249,'2015-07-20 13:43:26.960','2015-07-20 13:48:26.958','',0,null,'semaphore','',null,null,0,0)
          /*!*/;
          # at 155182967
          #150720 13:43:27 server id 3  end_log_pos 155182994     Xid = 8345994960
          COMMIT/*!*/;
          # at 155182994
          #150720 13:43:27 server id 3  end_log_pos 155183034     GTID 0-3-589791381 cid=8345994964
          /*!100001 SET @@session.gtid_seq_no=589791381*//*!*/;
          BEGIN
          /*!*/;
          # at 155183034
          #150720 13:43:27 server id 3  end_log_pos 155183186     Query   thread_id=34916277      exec_time=0     error_code=0
          SET TIMESTAMP=1437392607/*!*/;
          delete from `queueentry` where ((`OID` = 17490362) and (`OptimisticLockField` = 0))
          /*!*/;
          # at 155183186
          #150720 13:43:27 server id 3  end_log_pos 155183213     Xid = 8345994968
          COMMIT/*!*/;
          # at 155183213

          And then I finally got at least a single duplicate key error, where mark_start_commit is involved:

          MDEV8302: Deadlock kill GTID 0-3-589792742 (subid 590507973 in_commit=0) due to blocking GTID 0-3-589792741 (subid 590507972 in_commit=0)
          MDEV8302: mark_start_commit(GTID 0-3-589792742) after killed
          MDEV8302: Got deadlock kill in GTID 0-3-589792742 (subid 590507973 in_commit=1)
          MDEV8302: Retry #1 of GTID 0-3-589792742
          150720 14:13:53 [ERROR] Slave SQL: Error 'Duplicate entry 'semtest123456789_0-249' for key 'identry_idqueue'' on query. Default database: 'optovision'. Query: 'insert into `queueentry`(`identry`,`idqueue`,`e
          ntrydate`,`removedate`,`info`,`prio`,`reserved`,`ObjectContext`,`OriginHost`,`ReservingHost`,`UnreserveDate`,`ReservationCounter`,`OptimisticLockField`)values('semtest123456789_0',249,'2015-07-20 13:43:33.65
          2','2015-07-20 13:48:33.651','',0,null,'semaphore','',null,null,0,0)', Gtid 0-3-589792743, Internal MariaDB error code: 1062
          150720 14:13:53 [Warning] Slave: Duplicate entry 'semtest123456789_0-249' for key 'identry_idqueue' Error_code: 1062
          150720 14:13:53 [ERROR] Error running query, slave SQL thread aborted. Fix the problem, and restart the slave SQL thread with "SLAVE START". We stopped at log 'mysql-bin.007960' position 156039791; GTID posi
          tion '77-3-552,0-3-589792740,3-3-2,2-3-690710,17-3-180,1-3-7'
          150720 14:13:53 [ERROR] Slave (additional info): Commit failed due to failure of an earlier commit on which this one depends Error_code: 1964
          150720 14:13:53 [Warning] Slave: Commit failed due to failure of an earlier commit on which this one depends Error_code: 1964
          150720 14:13:53 [ERROR] Error running query, slave SQL thread aborted. Fix the problem, and restart the slave SQL thread with "SLAVE START". We stopped at log 'mysql-bin.007960' position 156040491; GTID posi
          tion '77-3-552,0-3-589792742,3-3-2,2-3-690710,17-3-180,1-3-7'
          150720 14:13:53 [ERROR] Slave (additional info): Commit failed due to failure of an earlier commit on which this one depends Error_code: 1964
          150720 14:13:53 [Warning] Slave: Commit failed due to failure of an earlier commit on which this one depends Error_code: 1964
          150720 14:13:53 [ERROR] Error running query, slave SQL thread aborted. Fix the problem, and restart the slave SQL thread with "SLAVE START". We stopped at log 'mysql-bin.007960' position 156040491; GTID posi
          tion '77-3-552,0-3-589792742,3-3-2,2-3-690710,17-3-180,1-3-7'
          150720 14:13:53 [Note] Slave SQL thread exiting, replication stopped in log 'mysql-bin.007960' at position 156040491; GTID position '77-3-552,0-3-589792742,3-3-2,2-3-690710,17-3-180,1-3-7'

          The binlog:

          #150720 13:43:34 server id 3  end_log_pos 156039831     GTID 0-3-589792741 cid=8346017507
          /*!100001 SET @@session.gtid_seq_no=589792741*//*!*/;
          BEGIN
          /*!*/;
          # at 156039831
          #150720 13:43:34 server id 3  end_log_pos 156039859     Intvar
          SET INSERT_ID=17491555/*!*/;
          # at 156039859
          #150720 13:43:34 server id 3  end_log_pos 156040245     Query   thread_id=34916452      exec_time=0     error_code=0
          SET TIMESTAMP=1437392614/*!*/;
          insert into `queueentry`(`identry`,`idqueue`,`entrydate`,`removedate`,`info`,`prio`,`reserved`,`ObjectContext`,`OriginHost`,`ReservingHost`,`UnreserveDate`,`ReservationCounter`,`OptimisticLockField`)values('
          semtest1233_0',249,'2015-07-20 13:43:33.639','2015-07-20 13:48:33.636','',0,null,'semaphore','',null,null,0,0)
          /*!*/;
          # at 156040245
          #150720 13:43:34 server id 3  end_log_pos 156040272     Xid = 8346017504
          COMMIT/*!*/;
          # at 156040272
          #150720 13:43:34 server id 3  end_log_pos 156040312     GTID 0-3-589792742 cid=8346017507
          /*!100001 SET @@session.gtid_seq_no=589792742*//*!*/;
          BEGIN
          /*!*/;
          # at 156040312
          #150720 13:43:34 server id 3  end_log_pos 156040464     Query   thread_id=34915524      exec_time=0     error_code=0
          SET TIMESTAMP=1437392614/*!*/;
          delete from `queueentry` where ((`OID` = 17491552) and (`OptimisticLockField` = 0))
          /*!*/;
          # at 156040464
          #150720 13:43:34 server id 3  end_log_pos 156040491     Xid = 8346017505
          COMMIT/*!*/;
          # at 156040491
          #150720 13:43:34 server id 3  end_log_pos 156040531     GTID 0-3-589792743 cid=8346017565
          /*!100001 SET @@session.gtid_seq_no=589792743*//*!*/;
          BEGIN
          /*!*/;
          # at 156040531
          #150720 13:43:34 server id 3  end_log_pos 156040559     Intvar
          SET INSERT_ID=17491557/*!*/;
          # at 156040559
          #150720 13:43:34 server id 3  end_log_pos 156040950     Query   thread_id=34915525      exec_time=0     error_code=0
          SET TIMESTAMP=1437392614/*!*/;
          insert into `queueentry`(`identry`,`idqueue`,`entrydate`,`removedate`,`info`,`prio`,`reserved`,`ObjectContext`,`OriginHost`,`ReservingHost`,`UnreserveDate`,`ReservationCounter`,`OptimisticLockField`)values('semtest123456789_0',249,'2015-07-20 13:43:33.652','2015-07-20 13:48:33.651','',0,null,'semaphore','',null,null,0,0)
          /*!*/;
          # at 156040950
          #150720 13:43:34 server id 3  end_log_pos 156040977     Xid = 8346017561
          COMMIT/*!*/;
          # at 156040977
          #150720 13:43:34 server id 3  end_log_pos 156041017     GTID 0-3-589792744 cid=8346017565
          /*!100001 SET @@session.gtid_seq_no=589792744*//*!*/;
          BEGIN
          /*!*/;
          # at 156041017
          #150720 13:43:34 server id 3  end_log_pos 156041169     Query   thread_id=34916452      exec_time=0     error_code=0
          SET TIMESTAMP=1437392614/*!*/;
          delete from `queueentry` where ((`OID` = 17491555) and (`OptimisticLockField` = 0))
          /*!*/;
          # at 156041169
          #150720 13:43:34 server id 3  end_log_pos 156041196     Xid = 8346017564
          COMMIT/*!*/;
          # at 156041196

          In all cases I made sure that the cids involved were not used by further statements before or after the excerpt from the binlog.

          As you can see from the name of the entry (including "test") we have configured our software to trigger this situation more frequent (but we were still not able to write a standalone program that crashes an empty database). So hopefully further tests should result in the problem faster.

          Now I need to know:

          • Is this the case we were looking for?
          • Do you have enough examples for now?
          • Shall we try to run the version including your fix again?

          Best regards,

          Thomas Mischke

          thomas.mischke Thomas Mischke added a comment - Sorry that it took so long to answer, I was sick a few days... But today I was able to reproduce the "duplicate key" errors, some without mark_start_commit involved, and one with mark_start_commit. We were using the version with just the logging, so the replication stopped in all cases, and using "slave start" the slave started again without further actions. Here are the details: MDEV8302: Deadlock kill GTID 0-3-589790787 (subid 590503582 in_commit=0) due to blocking GTID 0-3-589790786 (subid 590503581 in_commit=0) MDEV8302: Got deadlock kill in GTID 0-3-589790787 (subid 590503582 in_commit=1) MDEV8302: Retry #1 of GTID 0-3-589790787 150720 13:43:31 [ERROR] Slave SQL: Error 'Duplicate entry 'semtest123456789_0-249' for key 'identry_idqueue'' on query. Default database: 'optovision'. Query: 'insert into `queueentry`(`identry`,`idqueue`,`entrydate`,`removedate`,`info`,`prio`,`reserved`,`ObjectContext`,`OriginHost`,`ReservingHost`,`UnreserveDate`,`ReservationCounter`,`OptimisticLockField`)values('semtest123456789_0',249,'2015-07-20 13:43:23.477','2015-07-20 13:48:23.475','',0,null,'semaphore','',null,null,0,0)', Gtid 0-3-589790789, Internal MariaDB error code: 1062 150720 13:43:31 [Warning] Slave: Duplicate entry 'semtest123456789_0-249' for key 'identry_idqueue' Error_code: 1062 150720 13:43:31 [ERROR] Error running query, slave SQL thread aborted. Fix the problem, and restart the slave SQL thread with "SLAVE START". We stopped at log 'mysql-bin.007960' position 154941507; GTID position '77-3-552,0-3-589790785,3-3-2,2-3-690710,17-3-180,1-3-7' 150720 13:43:31 [Note] Slave SQL thread exiting, replication stopped in log 'mysql-bin.007960' at position 154942426; GTID position '77-3-552,0-3-589790788,3-3-2,2-3-690710,17-3-180,1-3-7' The binlog looks as follows: #150720 13:43:24 server id 3 end_log_pos 154941547 GTID 0-3-589790786 cid=8345984276 /*!100001 SET @@session.gtid_seq_no=589790786*//*!*/; BEGIN /*!*/; # at 154941547 #150720 13:43:24 server id 3 end_log_pos 154941575 Intvar SET INSERT_ID=17489759/*!*/; # at 154941575 #150720 13:43:24 server id 3 end_log_pos 154941961 Query thread_id=34916277 exec_time=0 error_code=0 SET TIMESTAMP=1437392604/*!*/; insert into `queueentry`(`identry`,`idqueue`,`entrydate`,`removedate`,`info`,`prio`,`reserved`,`ObjectContext`,`OriginHost`,`ReservingHost`,`UnreserveDate`,`ReservationCounter`,`OptimisticLockField`)values(' semtest1233_0',249,'2015-07-20 13:43:23.462','2015-07-20 13:48:23.460','',0,null,'semaphore','',null,null,0,0) /*!*/; # at 154941961 #150720 13:43:24 server id 3 end_log_pos 154941988 Xid = 8345984273 COMMIT/*!*/; # at 154941988 #150720 13:43:24 server id 3 end_log_pos 154942028 GTID 0-3-589790787 cid=8345984276 /*!100001 SET @@session.gtid_seq_no=589790787*//*!*/; BEGIN /*!*/; # at 154942028 #150720 13:43:24 server id 3 end_log_pos 154942180 Query thread_id=34915532 exec_time=0 error_code=0 SET TIMESTAMP=1437392604/*!*/; delete from `queueentry` where ((`OID` = 17489757) and (`OptimisticLockField` = 0)) /*!*/; # at 154942180 #150720 13:43:24 server id 3 end_log_pos 154942207 Xid = 8345984280 COMMIT/*!*/; # at 154942207 #150720 13:43:24 server id 3 end_log_pos 154942247 GTID 0-3-589790788 cid=8345984320 /*!100001 SET @@session.gtid_seq_no=589790788*//*!*/; BEGIN /*!*/; # at 154942247 #150720 13:43:24 server id 3 end_log_pos 154942399 Query thread_id=34916277 exec_time=0 error_code=0 SET TIMESTAMP=1437392604/*!*/; delete from `queueentry` where ((`OID` = 17489759) and (`OptimisticLockField` = 0)) /*!*/; # at 154942399 #150720 13:43:24 server id 3 end_log_pos 154942426 Xid = 8345984315 COMMIT/*!*/; # at 154942426 #150720 13:43:24 server id 3 end_log_pos 154942466 GTID 0-3-589790789 cid=8345984320 /*!100001 SET @@session.gtid_seq_no=589790789*//*!*/; BEGIN /*!*/; # at 154942466 #150720 13:43:24 server id 3 end_log_pos 154942494 Intvar SET INSERT_ID=17489761/*!*/; # at 154942494 #150720 13:43:24 server id 3 end_log_pos 154942885 Query thread_id=34915532 exec_time=0 error_code=0 SET TIMESTAMP=1437392604/*!*/; insert into `queueentry`(`identry`,`idqueue`,`entrydate`,`removedate`,`info`,`prio`,`reserved`,`ObjectContext`,`OriginHost`,`ReservingHost`,`UnreserveDate`,`ReservationCounter`,`OptimisticLockField`)values('semtest123456789_0',249,'2015-07-20 13:43:23.477','2015-07-20 13:48:23.475','',0,null,'semaphore','',null,null,0,0) /*!*/; # at 154942885 #150720 13:43:24 server id 3 end_log_pos 154942912 Xid = 8345984316 COMMIT/*!*/; # at 154942912 So in the first group we have: insert (some value) delete OID 17489757 In the second group we have: delete OID 17489759 insert semtest123456789_0-249 A little bit back in the binlog I found that OID 17489757 corresponds to semtest123456789_0-249: #150720 13:43:24 server id 3 end_log_pos 154940842 GTID 0-3-589790784 cid=8345984237 /*!100001 SET @@session.gtid_seq_no=589790784*//*!*/; BEGIN /*!*/; # at 154940842 #150720 13:43:24 server id 3 end_log_pos 154940870 Intvar SET INSERT_ID=17489757/*!*/; # at 154940870 #150720 13:43:24 server id 3 end_log_pos 154941261 Query thread_id=34915532 exec_time=0 error_code=0 SET TIMESTAMP=1437392604/*!*/; insert into `queueentry`(`identry`,`idqueue`,`entrydate`,`removedate`,`info`,`prio`,`reserved`,`ObjectContext`,`OriginHost`,`ReservingHost`,`UnreserveDate`,`ReservationCounter`,`OptimisticLockField`)values('semtest123456789_0',249,'2015-07-20 13:43:23.449','2015-07-20 13:48:23.448','',0,null,'semaphore','',null,null,0,0) /*!*/; # at 154941261 So the delete from group 1 gets killed due to the deadlock , and while the retry is in progress the insert from the second group starts to run and gets the duplicate key error. After starting the slave again it only took seconds to get the next error: MDEV8302: Deadlock kill GTID 0-3-589790816 (subid 590504252 in_commit=0) due to blocking GTID 0-3-589790815 (subid 590504251 in_commit=0) MDEV8302: Got deadlock kill in GTID 0-3-589790816 (subid 590504252 in_commit=1) MDEV8302: Retry #1 of GTID 0-3-589790816 150720 13:58:03 [ERROR] Slave SQL: Error 'Duplicate entry 'semtest123456789_0-249' for key 'identry_idqueue'' on query. Default database: 'optovision'. Query: 'insert into `queueentry`(`identry`,`idqueue`,`entrydate`,`removedate`,`info`,`prio`,`reserved`,`ObjectContext`,`OriginHost`,`ReservingHost`,`UnreserveDate`,`ReservationCounter`,`OptimisticLockField`)values('semtest123456789_0',249,'2015-07-20 13:43:23.674','2015-07-20 13:48:23.673','',0,null,'semaphore','',null,null,0,0)', Gtid 0-3-589790818, Internal MariaDB error code: 1062 150720 13:58:03 [Warning] Slave: Duplicate entry 'semtest123456789_0-249' for key 'identry_idqueue' Error_code: 1062 MDEV8302: Deadlock kill GTID 0-3-589790818 (subid 590504254 in_commit=0) due to blocking GTID 0-3-589790816 (subid 590504252 in_commit=0) 150720 13:58:03 [ERROR] Error running query, slave SQL thread aborted. Fix the problem, and restart the slave SQL thread with "SLAVE START". We stopped at log 'mysql-bin.007960' position 154952396; GTID position '77-3-552,0-3-589790815,3-3-2,2-3-690710,17-3-180,1-3-7' 150720 13:58:03 [Note] Slave SQL thread exiting, replication stopped in log 'mysql-bin.007960' at position 154952834; GTID position '77-3-552,0-3-589790817,3-3-2,2-3-690710,17-3-180,1-3-7' In this case the log messages are in a different order. The binlog looks as follows: #150720 13:43:24 server id 3 end_log_pos 154951955 GTID 0-3-589790815 cid=8345984957 /*!100001 SET @@session.gtid_seq_no=589790815*//*!*/; BEGIN /*!*/; # at 154951955 #150720 13:43:24 server id 3 end_log_pos 154951983 Intvar SET INSERT_ID=17489795/*!*/; # at 154951983 #150720 13:43:24 server id 3 end_log_pos 154952369 Query thread_id=34916277 exec_time=0 error_code=0 SET TIMESTAMP=1437392604/*!*/; insert into `queueentry`(`identry`,`idqueue`,`entrydate`,`removedate`,`info`,`prio`,`reserved`,`ObjectContext`,`OriginHost`,`ReservingHost`,`UnreserveDate`,`ReservationCounter`,`OptimisticLockField`)values(' semtest1233_0',249,'2015-07-20 13:43:23.660','2015-07-20 13:48:23.659','',0,null,'semaphore','',null,null,0,0) /*!*/; # at 154952369 #150720 13:43:24 server id 3 end_log_pos 154952396 Xid = 8345984954 COMMIT/*!*/; # at 154952396 #150720 13:43:24 server id 3 end_log_pos 154952436 GTID 0-3-589790816 cid=8345984957 /*!100001 SET @@session.gtid_seq_no=589790816*//*!*/; BEGIN /*!*/; # at 154952436 #150720 13:43:24 server id 3 end_log_pos 154952588 Query thread_id=34915532 exec_time=0 error_code=0 SET TIMESTAMP=1437392604/*!*/; delete from `queueentry` where ((`OID` = 17489792) and (`OptimisticLockField` = 0)) /*!*/; # at 154952588 #150720 13:43:24 server id 3 end_log_pos 154952615 Xid = 8345984965 COMMIT/*!*/; # at 154952615 #150720 13:43:24 server id 3 end_log_pos 154952655 GTID 0-3-589790817 cid=8345985010 /*!100001 SET @@session.gtid_seq_no=589790817*//*!*/; BEGIN /*!*/; # at 154952655 #150720 13:43:24 server id 3 end_log_pos 154952807 Query thread_id=34916277 exec_time=0 error_code=0 SET TIMESTAMP=1437392604/*!*/; delete from `queueentry` where ((`OID` = 17489795) and (`OptimisticLockField` = 0)) /*!*/; # at 154952807 #150720 13:43:24 server id 3 end_log_pos 154952834 Xid = 8345985005 COMMIT/*!*/; # at 154952834 #150720 13:43:24 server id 3 end_log_pos 154952874 GTID 0-3-589790818 cid=8345985010 /*!100001 SET @@session.gtid_seq_no=589790818*//*!*/; BEGIN /*!*/; # at 154952874 #150720 13:43:24 server id 3 end_log_pos 154952902 Intvar SET INSERT_ID=17489797/*!*/; # at 154952902 #150720 13:43:24 server id 3 end_log_pos 154953293 Query thread_id=34915524 exec_time=0 error_code=0 SET TIMESTAMP=1437392604/*!*/; insert into `queueentry`(`identry`,`idqueue`,`entrydate`,`removedate`,`info`,`prio`,`reserved`,`ObjectContext`,`OriginHost`,`ReservingHost`,`UnreserveDate`,`ReservationCounter`,`OptimisticLockField`)values('semtest123456789_0',249,'2015-07-20 13:43:23.674','2015-07-20 13:48:23.673','',0,null,'semaphore','',null,null,0,0) /*!*/; # at 154953293 #150720 13:43:24 server id 3 end_log_pos 154953320 Xid = 8345985004 COMMIT/*!*/; # at 154953320 Group 1 has two statements: insert (some value) delete OID 17489792 Group 2 has: delete insert semtest123456789_0-249 Again the OID 17489792 matches the key value of the insert (semtest123456789_0-249). Again I restarted the slave, and the next error looks as follows (again different logging): MDEV8302: Deadlock kill GTID 0-3-589791379 (subid 590505427 in_commit=0) due to blocking GTID 0-3-589791378 (subid 590505426 in_commit=0) MDEV8302: Got deadlock kill in GTID 0-3-589791379 (subid 590505427 in_commit=1) MDEV8302: Retry #1 of GTID 0-3-589791379 150720 14:09:48 [ERROR] Slave SQL: Error 'Duplicate entry 'semtest123456789_0-249' for key 'identry_idqueue'' on query. Default database: 'optovision'. Query: 'insert into `queueentry`(`identry`,`idqueue`,`entrydate`,`removedate`,`info`,`prio`,`reserved`,`ObjectContext`,`OriginHost`,`ReservingHost`,`UnreserveDate`,`ReservationCounter`,`OptimisticLockField`)values('semtest123456789_0',249,'2015-07-20 13:43:26.960','2015-07-20 13:48:26.958','',0,null,'semaphore','',null,null,0,0)', Gtid 0-3-589791380, Internal MariaDB error code: 1062 150720 14:09:48 [Warning] Slave: Duplicate entry 'semtest123456789_0-249' for key 'identry_idqueue' Error_code: 1062 150720 14:09:48 [ERROR] Error running query, slave SQL thread aborted. Fix the problem, and restart the slave SQL thread with "SLAVE START". We stopped at log 'mysql-bin.007960' position 155182289; GTID position '77-3-552,0-3-589791378,3-3-2,2-3-690710,17-3-180,1-3-7' 150720 14:09:48 [ERROR] Slave (additional info): Commit failed due to failure of an earlier commit on which this one depends Error_code: 1964 150720 14:09:48 [Warning] Slave: Commit failed due to failure of an earlier commit on which this one depends Error_code: 1964 150720 14:09:48 [ERROR] Error running query, slave SQL thread aborted. Fix the problem, and restart the slave SQL thread with "SLAVE START". We stopped at log 'mysql-bin.007960' position 155182508; GTID position '77-3-552,0-3-589791379,3-3-2,2-3-690710,17-3-180,1-3-7' 150720 14:09:48 [ERROR] Slave (additional info): Commit failed due to failure of an earlier commit on which this one depends Error_code: 1964 150720 14:09:48 [Warning] Slave: Commit failed due to failure of an earlier commit on which this one depends Error_code: 1964 150720 14:09:48 [ERROR] Error running query, slave SQL thread aborted. Fix the problem, and restart the slave SQL thread with "SLAVE START". We stopped at log 'mysql-bin.007960' position 155182508; GTID position '77-3-552,0-3-589791379,3-3-2,2-3-690710,17-3-180,1-3-7' 150720 14:09:48 [ERROR] Slave (additional info): Commit failed due to failure of an earlier commit on which this one depends Error_code: 1964 150720 14:09:48 [Warning] Slave: Commit failed due to failure of an earlier commit on which this one depends Error_code: 1964 150720 14:09:48 [ERROR] Error running query, slave SQL thread aborted. Fix the problem, and restart the slave SQL thread with "SLAVE START". We stopped at log 'mysql-bin.007960' position 155182508; GTID position '77-3-552,0-3-589791379,3-3-2,2-3-690710,17-3-180,1-3-7' 150720 14:09:48 [Note] Slave SQL thread exiting, replication stopped in log 'mysql-bin.007960' at position 155182508; GTID position '77-3-552,0-3-589791379,3-3-2,2-3-690710,17-3-180,1-3-7' 150720 14:10:02 [ERROR] Error reading packet from server: Lost connection to MySQL server during query ( server_errno=2013) 150720 14:10:02 [Note] Slave I/O thread killed while reading event 150720 14:10:02 [Note] Slave I/O thread exiting, read up to log 'mysql-bin.007960', position 263021432; GTID position 77-3-552,0-3-589886520,3-3-2,2-3-690710,1-3-7,17-3-180 Again the binlog: #150720 13:43:27 server id 3 end_log_pos 155181848 GTID 0-3-589791378 cid=8345994929 /*!100001 SET @@session.gtid_seq_no=589791378*//*!*/; BEGIN /*!*/; # at 155181848 #150720 13:43:27 server id 3 end_log_pos 155181876 Intvar SET INSERT_ID=17490362/*!*/; # at 155181876 #150720 13:43:27 server id 3 end_log_pos 155182262 Query thread_id=34916277 exec_time=0 error_code=0 SET TIMESTAMP=1437392607/*!*/; insert into `queueentry`(`identry`,`idqueue`,`entrydate`,`removedate`,`info`,`prio`,`reserved`,`ObjectContext`,`OriginHost`,`ReservingHost`,`UnreserveDate`,`ReservationCounter`,`OptimisticLockField`)values(' semtest1233_0',249,'2015-07-20 13:43:26.946','2015-07-20 13:48:26.945','',0,null,'semaphore','',null,null,0,0) /*!*/; # at 155182262 #150720 13:43:27 server id 3 end_log_pos 155182289 Xid = 8345994927 COMMIT/*!*/; # at 155182289 #150720 13:43:27 server id 3 end_log_pos 155182329 GTID 0-3-589791379 cid=8345994929 /*!100001 SET @@session.gtid_seq_no=589791379*//*!*/; BEGIN /*!*/; # at 155182329 #150720 13:43:27 server id 3 end_log_pos 155182481 Query thread_id=34915525 exec_time=0 error_code=0 SET TIMESTAMP=1437392607/*!*/; delete from `queueentry` where ((`OID` = 17490359) and (`OptimisticLockField` = 0)) /*!*/; # at 155182481 #150720 13:43:27 server id 3 end_log_pos 155182508 Xid = 8345994933 COMMIT/*!*/; # at 155182508 #150720 13:43:27 server id 3 end_log_pos 155182548 GTID 0-3-589791380 cid=8345994964 /*!100001 SET @@session.gtid_seq_no=589791380*//*!*/; BEGIN /*!*/; # at 155182548 #150720 13:43:27 server id 3 end_log_pos 155182576 Intvar SET INSERT_ID=17490364/*!*/; # at 155182576 #150720 13:43:27 server id 3 end_log_pos 155182967 Query thread_id=34915524 exec_time=0 error_code=0 SET TIMESTAMP=1437392607/*!*/; insert into `queueentry`(`identry`,`idqueue`,`entrydate`,`removedate`,`info`,`prio`,`reserved`,`ObjectContext`,`OriginHost`,`ReservingHost`,`UnreserveDate`,`ReservationCounter`,`OptimisticLockField`)values('semtest123456789_0',249,'2015-07-20 13:43:26.960','2015-07-20 13:48:26.958','',0,null,'semaphore','',null,null,0,0) /*!*/; # at 155182967 #150720 13:43:27 server id 3 end_log_pos 155182994 Xid = 8345994960 COMMIT/*!*/; # at 155182994 #150720 13:43:27 server id 3 end_log_pos 155183034 GTID 0-3-589791381 cid=8345994964 /*!100001 SET @@session.gtid_seq_no=589791381*//*!*/; BEGIN /*!*/; # at 155183034 #150720 13:43:27 server id 3 end_log_pos 155183186 Query thread_id=34916277 exec_time=0 error_code=0 SET TIMESTAMP=1437392607/*!*/; delete from `queueentry` where ((`OID` = 17490362) and (`OptimisticLockField` = 0)) /*!*/; # at 155183186 #150720 13:43:27 server id 3 end_log_pos 155183213 Xid = 8345994968 COMMIT/*!*/; # at 155183213 And then I finally got at least a single duplicate key error, where mark_start_commit is involved: MDEV8302: Deadlock kill GTID 0-3-589792742 (subid 590507973 in_commit=0) due to blocking GTID 0-3-589792741 (subid 590507972 in_commit=0) MDEV8302: mark_start_commit(GTID 0-3-589792742) after killed MDEV8302: Got deadlock kill in GTID 0-3-589792742 (subid 590507973 in_commit=1) MDEV8302: Retry #1 of GTID 0-3-589792742 150720 14:13:53 [ERROR] Slave SQL: Error 'Duplicate entry 'semtest123456789_0-249' for key 'identry_idqueue'' on query. Default database: 'optovision'. Query: 'insert into `queueentry`(`identry`,`idqueue`,`e ntrydate`,`removedate`,`info`,`prio`,`reserved`,`ObjectContext`,`OriginHost`,`ReservingHost`,`UnreserveDate`,`ReservationCounter`,`OptimisticLockField`)values('semtest123456789_0',249,'2015-07-20 13:43:33.65 2','2015-07-20 13:48:33.651','',0,null,'semaphore','',null,null,0,0)', Gtid 0-3-589792743, Internal MariaDB error code: 1062 150720 14:13:53 [Warning] Slave: Duplicate entry 'semtest123456789_0-249' for key 'identry_idqueue' Error_code: 1062 150720 14:13:53 [ERROR] Error running query, slave SQL thread aborted. Fix the problem, and restart the slave SQL thread with "SLAVE START". We stopped at log 'mysql-bin.007960' position 156039791; GTID posi tion '77-3-552,0-3-589792740,3-3-2,2-3-690710,17-3-180,1-3-7' 150720 14:13:53 [ERROR] Slave (additional info): Commit failed due to failure of an earlier commit on which this one depends Error_code: 1964 150720 14:13:53 [Warning] Slave: Commit failed due to failure of an earlier commit on which this one depends Error_code: 1964 150720 14:13:53 [ERROR] Error running query, slave SQL thread aborted. Fix the problem, and restart the slave SQL thread with "SLAVE START". We stopped at log 'mysql-bin.007960' position 156040491; GTID posi tion '77-3-552,0-3-589792742,3-3-2,2-3-690710,17-3-180,1-3-7' 150720 14:13:53 [ERROR] Slave (additional info): Commit failed due to failure of an earlier commit on which this one depends Error_code: 1964 150720 14:13:53 [Warning] Slave: Commit failed due to failure of an earlier commit on which this one depends Error_code: 1964 150720 14:13:53 [ERROR] Error running query, slave SQL thread aborted. Fix the problem, and restart the slave SQL thread with "SLAVE START". We stopped at log 'mysql-bin.007960' position 156040491; GTID posi tion '77-3-552,0-3-589792742,3-3-2,2-3-690710,17-3-180,1-3-7' 150720 14:13:53 [Note] Slave SQL thread exiting, replication stopped in log 'mysql-bin.007960' at position 156040491; GTID position '77-3-552,0-3-589792742,3-3-2,2-3-690710,17-3-180,1-3-7' The binlog: #150720 13:43:34 server id 3 end_log_pos 156039831 GTID 0-3-589792741 cid=8346017507 /*!100001 SET @@session.gtid_seq_no=589792741*//*!*/; BEGIN /*!*/; # at 156039831 #150720 13:43:34 server id 3 end_log_pos 156039859 Intvar SET INSERT_ID=17491555/*!*/; # at 156039859 #150720 13:43:34 server id 3 end_log_pos 156040245 Query thread_id=34916452 exec_time=0 error_code=0 SET TIMESTAMP=1437392614/*!*/; insert into `queueentry`(`identry`,`idqueue`,`entrydate`,`removedate`,`info`,`prio`,`reserved`,`ObjectContext`,`OriginHost`,`ReservingHost`,`UnreserveDate`,`ReservationCounter`,`OptimisticLockField`)values(' semtest1233_0',249,'2015-07-20 13:43:33.639','2015-07-20 13:48:33.636','',0,null,'semaphore','',null,null,0,0) /*!*/; # at 156040245 #150720 13:43:34 server id 3 end_log_pos 156040272 Xid = 8346017504 COMMIT/*!*/; # at 156040272 #150720 13:43:34 server id 3 end_log_pos 156040312 GTID 0-3-589792742 cid=8346017507 /*!100001 SET @@session.gtid_seq_no=589792742*//*!*/; BEGIN /*!*/; # at 156040312 #150720 13:43:34 server id 3 end_log_pos 156040464 Query thread_id=34915524 exec_time=0 error_code=0 SET TIMESTAMP=1437392614/*!*/; delete from `queueentry` where ((`OID` = 17491552) and (`OptimisticLockField` = 0)) /*!*/; # at 156040464 #150720 13:43:34 server id 3 end_log_pos 156040491 Xid = 8346017505 COMMIT/*!*/; # at 156040491 #150720 13:43:34 server id 3 end_log_pos 156040531 GTID 0-3-589792743 cid=8346017565 /*!100001 SET @@session.gtid_seq_no=589792743*//*!*/; BEGIN /*!*/; # at 156040531 #150720 13:43:34 server id 3 end_log_pos 156040559 Intvar SET INSERT_ID=17491557/*!*/; # at 156040559 #150720 13:43:34 server id 3 end_log_pos 156040950 Query thread_id=34915525 exec_time=0 error_code=0 SET TIMESTAMP=1437392614/*!*/; insert into `queueentry`(`identry`,`idqueue`,`entrydate`,`removedate`,`info`,`prio`,`reserved`,`ObjectContext`,`OriginHost`,`ReservingHost`,`UnreserveDate`,`ReservationCounter`,`OptimisticLockField`)values('semtest123456789_0',249,'2015-07-20 13:43:33.652','2015-07-20 13:48:33.651','',0,null,'semaphore','',null,null,0,0) /*!*/; # at 156040950 #150720 13:43:34 server id 3 end_log_pos 156040977 Xid = 8346017561 COMMIT/*!*/; # at 156040977 #150720 13:43:34 server id 3 end_log_pos 156041017 GTID 0-3-589792744 cid=8346017565 /*!100001 SET @@session.gtid_seq_no=589792744*//*!*/; BEGIN /*!*/; # at 156041017 #150720 13:43:34 server id 3 end_log_pos 156041169 Query thread_id=34916452 exec_time=0 error_code=0 SET TIMESTAMP=1437392614/*!*/; delete from `queueentry` where ((`OID` = 17491555) and (`OptimisticLockField` = 0)) /*!*/; # at 156041169 #150720 13:43:34 server id 3 end_log_pos 156041196 Xid = 8346017564 COMMIT/*!*/; # at 156041196 In all cases I made sure that the cids involved were not used by further statements before or after the excerpt from the binlog. As you can see from the name of the entry (including "test") we have configured our software to trigger this situation more frequent (but we were still not able to write a standalone program that crashes an empty database). So hopefully further tests should result in the problem faster. Now I need to know: Is this the case we were looking for? Do you have enough examples for now? Shall we try to run the version including your fix again? Best regards, Thomas Mischke

          Thanks for your continued help with tracking this down, it is much
          appreciated. It is good that you are now able to trigger it more frequently.

          In all your cases, we see that the deadlock is detected by T1 while T2 has
          still not reached its commit stage:

          MDEV8302: Deadlock kill T2 (... in_commit=0) due to blocking T1 (... in_commit=0)

          But when T2 catches the error, it has started to commit:

          MDEV8302: Got deadlock kill in T2 (... in_commit=1)

          This is interesting, because it suggests that the condition that triggers
          the deadlock is temporary, and resolves itself when T1 reaches commit. If
          this is right, then there is no real deadlock, and we could end up
          incorrectly killing T2 even though T1 is not blocked. This could explain the
          duplicate key error. However, the absense in some cases of the printout
          "mark_start_commit(T2) after killed" seems to indicate that this is not
          the case, and when I asked an InnoDB developer, he also thought it was not
          possible. So we still do not know .

          I have prepared a new patch with extra printouts for you:

          http://lists.askmonty.org/pipermail/commits/2015-July/008197.html

          With this patch, you should enable --innodb-print_all_deadlocks=1 (on just
          your test slave). Then, whenever a deadlock is detected, we will get
          detailed information about which locks are involved in the error log. There
          is also an additional printout ("GTID XXX unmark after signalling next"),
          which will confirm if the problem really is starting the following
          transactions too early, as we suspect.

          If you can trigger the bug with this new patch running with
          --innodb-print_all_deadlocks=1, and supply the info in the error log as
          before, then that might help proceed further - at least it is my best idea,
          failing a method to reliably reproduce the deadlock condition.

          Another thing you could do is to try again running the old version
          including my potential fix. If this makes the duplicate key error go away,
          then that is surely interesting (and the fix should be valid in any
          case). Given that the "mark_start_commit(T2) after killed" was absent in
          some of the failures, it might be that the fix is not effective, but it is
          hard to tell for sure given our lack of understanding of the details of the
          problem, and knowing for sure will be helpful either way.

          knielsen Kristian Nielsen added a comment - Thanks for your continued help with tracking this down, it is much appreciated. It is good that you are now able to trigger it more frequently. In all your cases, we see that the deadlock is detected by T1 while T2 has still not reached its commit stage: MDEV8302: Deadlock kill T2 (... in_commit=0) due to blocking T1 (... in_commit=0) But when T2 catches the error, it has started to commit: MDEV8302: Got deadlock kill in T2 (... in_commit=1) This is interesting, because it suggests that the condition that triggers the deadlock is temporary, and resolves itself when T1 reaches commit. If this is right, then there is no real deadlock, and we could end up incorrectly killing T2 even though T1 is not blocked. This could explain the duplicate key error. However, the absense in some cases of the printout "mark_start_commit(T2) after killed" seems to indicate that this is not the case, and when I asked an InnoDB developer, he also thought it was not possible. So we still do not know . I have prepared a new patch with extra printouts for you: http://lists.askmonty.org/pipermail/commits/2015-July/008197.html With this patch, you should enable --innodb-print_all_deadlocks=1 (on just your test slave). Then, whenever a deadlock is detected, we will get detailed information about which locks are involved in the error log. There is also an additional printout ("GTID XXX unmark after signalling next"), which will confirm if the problem really is starting the following transactions too early, as we suspect. If you can trigger the bug with this new patch running with --innodb-print_all_deadlocks=1, and supply the info in the error log as before, then that might help proceed further - at least it is my best idea, failing a method to reliably reproduce the deadlock condition. Another thing you could do is to try again running the old version including my potential fix. If this makes the duplicate key error go away, then that is surely interesting (and the fix should be valid in any case). Given that the "mark_start_commit(T2) after killed" was absent in some of the failures, it might be that the fix is not effective, but it is hard to tell for sure given our lack of understanding of the details of the problem, and knowing for sure will be helpful either way.

          Here we go. I compiled the new version, added the innodb_print_all_deadlocks and reproduced the problem.

          Here is the output of the log file. Just for completeness: There are much more messages about MDEV8302 and deadlocks, but they all belong to transactions long before the deadlock, so I do not report them.

          MDEV8302: Deadlock kill GTID 0-3-616241346 (subid 617021915 in_commit=0) due to blocking GTID 0-3-616241345 (subid 617021914 in_commit=0)
          MDEV8302: mark_start_commit(GTID 0-3-616241346) after killed
          MDEV8302: InnoDB deadlock kill trx_id=11014855138 lock type=32 mode=3
          TRANSACTION 11014855138, ACTIVE 0 sec rollback
          ROLLING BACK 3 lock struct(s), heap size 1184, 2 row lock(s)
          MySQL thread id 36, OS thread handle 0x7f9424d91700, query id 88346 closing tables
          RECORD LOCKS space id 4236271 page no 2329 n bits 440 index `identry_idqueue` of table `optovision`.`queueentry` trx table locks 1 total table locks 2  trx id 11014855138 lock_mode X locks rec but not gap lock hold time 0 wait time before grant 0
          MDEV8302: Got deadlock kill in GTID 0-3-616241346 (subid 617021915 in_commit=1)
          MDEV8302: Retry #1 of GTID 0-3-616241346
          150729 10:36:50 [ERROR] Slave SQL: Error 'Duplicate entry 'semtest123456789_0-249' for key 'identry_idqueue'' on query. Default database: 'optovision'. Query: 'insert into `queueentry`(`identry`,`idqueue`,`entrydate`,`removedate`,`info`,`prio`,`reserved`,`ObjectContext`,`OriginHost`,`ReservingHost`,`UnreserveDate`,`ReservationCounter`,`OptimisticLockField`)values('semtest123456789_0',249,'2015-07-29 10:36:24.067','2015-07-29 10:41:24.066','',0,null,'semaphore','',null,null,0,0)', Gtid 0-3-616241347, Internal MariaDB error code: 1062
          MDEV8302: unmark_start_commit(GTID 0-3-616241346) count_committing=4199 next->wait_count=4199
          MDEV8302:   GTID 0-3-616241346 unmark after signalling next, count_committing=4199 next->wait_count=4199 next->installed=1
          150729 10:36:50 [Warning] Slave: Duplicate entry 'semtest123456789_0-249' for key 'identry_idqueue' Error_code: 1062
          150729 10:36:50 [ERROR] Error running query, slave SQL thread aborted. Fix the problem, and restart the slave SQL thread with "SLAVE START". We stopped at log 'mysql-bin.007999' position 50282813; GTID position '77-3-552,0-3-616241344,3-3-2,2-3-733822,17-3-180,1-3-7'
          150729 10:36:50 [ERROR] Slave (additional info): Commit failed due to failure of an earlier commit on which this one depends Error_code: 1964
          150729 10:36:50 [Warning] Slave: Commit failed due to failure of an earlier commit on which this one depends Error_code: 1964
          150729 10:36:50 [ERROR] Error running query, slave SQL thread aborted. Fix the problem, and restart the slave SQL thread with "SLAVE START". We stopped at log 'mysql-bin.007999' position 50283513; GTID position '77-3-552,0-3-616241346,3-3-2,2-3-733822,17-3-180,1-3-7'
          150729 10:36:50 [ERROR] Slave (additional info): Commit failed due to failure of an earlier commit on which this one depends Error_code: 1964
          150729 10:36:50 [Warning] Slave: Commit failed due to failure of an earlier commit on which this one depends Error_code: 1964
          150729 10:36:50 [ERROR] Error running query, slave SQL thread aborted. Fix the problem, and restart the slave SQL thread with "SLAVE START". We stopped at log 'mysql-bin.007999' position 50283513; GTID position '77-3-552,0-3-616241346,3-3-2,2-3-733822,17-3-180,1-3-7'
          150729 10:36:50 [ERROR] Slave (additional info): Commit failed due to failure of an earlier commit on which this one depends Error_code: 1964
          150729 10:36:50 [Warning] Slave: Commit failed due to failure of an earlier commit on which this one depends Error_code: 1964
          150729 10:36:50 [ERROR] Error running query, slave SQL thread aborted. Fix the problem, and restart the slave SQL thread with "SLAVE START". We stopped at log 'mysql-bin.007999' position 50283513; GTID position '77-3-552,0-3-616241346,3-3-2,2-3-733822,17-3-180,1-3-7'
          150729 10:36:50 [ERROR] Slave (additional info): Commit failed due to failure of an earlier commit on which this one depends Error_code: 1964
          150729 10:36:50 [Warning] Slave: Duplicate entry 'semtest123456789_0-249' for key 'identry_idqueue' Error_code: 1062
          150729 10:36:50 [Warning] Slave: Commit failed due to failure of an earlier commit on which this one depends Error_code: 1964
          150729 10:36:50 [ERROR] Error running query, slave SQL thread aborted. Fix the problem, and restart the slave SQL thread with "SLAVE START". We stopped at log 'mysql-bin.007999' position 50283513; GTID position '77-3-552,0-3-616241346,3-3-2,2-3-733822,17-3-180,1-3-7'
          150729 10:36:50 [Note] Slave SQL thread exiting, replication stopped in log 'mysql-bin.007999' at position 50283513; GTID position '77-3-552,0-3-616241346,3-3-2,2-3-733822,17-3-180,1-3-7'

          The binlog of the master looks as folows (again I made sure that the delete of the first group deletes the key from the insert in the second group; further I checked that no MDEV8302 or deadlocks were reported when this key was written first):

          #150729 10:36:41 server id 3  end_log_pos 50282853      GTID 0-3-616241345 cid=8871685554
          /*!100001 SET @@session.gtid_seq_no=616241345*//*!*/;
          BEGIN
          /*!*/;
          # at 50282853
          #150729 10:36:41 server id 3  end_log_pos 50282881      Intvar
          SET INSERT_ID=19563752/*!*/;
          # at 50282881
          #150729 10:36:41 server id 3  end_log_pos 50283267      Query   thread_id=36782520      exec_time=0     error_code=0
          SET TIMESTAMP=1438159001/*!*/;
          insert into `queueentry`(`identry`,`idqueue`,`entrydate`,`removedate`,`info`,`prio`,`reserved`,`ObjectContext`,`OriginHost`,`ReservingHost`,`UnreserveDate`,`ReservationCounter`,`OptimisticLockField`)values('
          semtest1233_0',249,'2015-07-29 10:36:24.053','2015-07-29 10:41:24.052','',0,null,'semaphore','',null,null,0,0)
          /*!*/;
          # at 50283267
          #150729 10:36:41 server id 3  end_log_pos 50283294      Xid = 8871685551
          COMMIT/*!*/;
          # at 50283294
          #150729 10:36:41 server id 3  end_log_pos 50283334      GTID 0-3-616241346 cid=8871685554
          /*!100001 SET @@session.gtid_seq_no=616241346*//*!*/;
          BEGIN
          /*!*/;
          # at 50283334
          #150729 10:36:41 server id 3  end_log_pos 50283486      Query   thread_id=36782495      exec_time=0     error_code=0
          SET TIMESTAMP=1438159001/*!*/;
          delete from `queueentry` where ((`OID` = 19563749) and (`OptimisticLockField` = 0))
          /*!*/;
          # at 50283486
          #150729 10:36:41 server id 3  end_log_pos 50283513      Xid = 8871685553
          COMMIT/*!*/;
          # at 50283513
          #150729 10:36:41 server id 3  end_log_pos 50283553      GTID 0-3-616241347 cid=8871685582
          /*!100001 SET @@session.gtid_seq_no=616241347*//*!*/;
          BEGIN
          /*!*/;
          # at 50283553
          #150729 10:36:41 server id 3  end_log_pos 50283581      Intvar
          SET INSERT_ID=19563754/*!*/;
          # at 50283581
          #150729 10:36:41 server id 3  end_log_pos 50283972      Query   thread_id=36782519      exec_time=0     error_code=0
          SET TIMESTAMP=1438159001/*!*/;
          insert into `queueentry`(`identry`,`idqueue`,`entrydate`,`removedate`,`info`,`prio`,`reserved`,`ObjectContext`,`OriginHost`,`ReservingHost`,`UnreserveDate`,`ReservationCounter`,`OptimisticLockField`)values('semtest123456789_0',249,'2015-07-29 10:36:24.067','2015-07-29 10:41:24.066','',0,null,'semaphore','',null,null,0,0)
          /*!*/;
          # at 50283972
          #150729 10:36:41 server id 3  end_log_pos 50283999      Xid = 8871685578
          COMMIT/*!*/;
          # at 50283999
          #150729 10:36:41 server id 3  end_log_pos 50284039      GTID 0-3-616241348 cid=8871685582
          /*!100001 SET @@session.gtid_seq_no=616241348*//*!*/;
          BEGIN
          /*!*/;
          # at 50284039
          #150729 10:36:41 server id 3  end_log_pos 50284191      Query   thread_id=36782520      exec_time=0     error_code=0
          SET TIMESTAMP=1438159001/*!*/;
          delete from `queueentry` where ((`OID` = 19563752) and (`OptimisticLockField` = 0))
          /*!*/;
          # at 50284191
          #150729 10:36:41 server id 3  end_log_pos 50284218      Xid = 8871685585
          COMMIT/*!*/;
          # at 50284218

          Here is a second case, this time without mark_start_commit:

          The log file:

          MDEV8302: Deadlock kill GTID 0-3-616369406 (subid 617151562 in_commit=0) due to blocking GTID 0-3-616369405 (subid 617151561 in_commit=0)
          MDEV8302: InnoDB deadlock kill trx_id=11015095365 lock type=32 mode=3
          TRANSACTION 11015095365, ACTIVE 0 sec rollback
          ROLLING BACK 3 lock struct(s), heap size 1184, 2 row lock(s)
          MySQL thread id 270, OS thread handle 0x7f918e119700, query id 455867 closing tables
          RECORD LOCKS space id 4236271 page no 2353 n bits 312 index `identry_idqueue` of table `optovision`.`queueentry` trx table locks 1 total table locks 2  trx id 11015095365 lock_mode X locks rec but not gap lock hold time 0 wait time before grant 0
          MDEV8302: Got deadlock kill in GTID 0-3-616369406 (subid 617151562 in_commit=1)
          MDEV8302: Retry #1 of GTID 0-3-616369406
          MDEV8302: unmark_start_commit(GTID 0-3-616369406) count_committing=127 next->wait_count=127
          MDEV8302:   GTID 0-3-616369406 unmark after signalling next, count_committing=127 next->wait_count=127 next->installed=1
          150729 11:00:23 [ERROR] Slave SQL: Error 'Duplicate entry 'semtest123456789_0-249' for key 'identry_idqueue'' on query. Default database: 'optovision'. Query: 'insert into `queueentry`(`identry`,`idqueue`,`entrydate`,`removedate`,`info`,`prio`,`reserved`,`ObjectContext`,`OriginHost`,`ReservingHost`,`UnreserveDate`,`ReservationCounter`,`OptimisticLockField`)values('semtest123456789_0',249,'2015-07-29 10:59:30.599','2015-07-29 11:04:30.598','',0,null,'semaphore','',null,null,0,0)', Gtid 0-3-616369408, Internal MariaDB error code: 1062
          150729 11:00:23 [Warning] Slave: Duplicate entry 'semtest123456789_0-249' for key 'identry_idqueue' Error_code: 1062
          150729 11:00:23 [ERROR] Error running query, slave SQL thread aborted. Fix the problem, and restart the slave SQL thread with "SLAVE START". We stopped at log 'mysql-bin.007999' position 202834844; GTID position '77-3-552,0-3-616369404,3-3-2,2-3-733822,17-3-180,1-3-7'
          150729 11:00:23 [Note] Slave SQL thread exiting, replication stopped in log 'mysql-bin.007999' at position 202835763; GTID position '77-3-552,0-3-616369407,3-3-2,2-3-733822,17-3-180,1-3-7'

          And the binlog of the master:

          #150729 10:59:48 server id 3  end_log_pos 202834884     GTID 0-3-616369405 cid=8873293914
          /*!100001 SET @@session.gtid_seq_no=616369405*//*!*/;
          BEGIN
          /*!*/;
          # at 202834884
          #150729 10:59:48 server id 3  end_log_pos 202834912     Intvar
          SET INSERT_ID=19599033/*!*/;
          # at 202834912
          #150729 10:59:48 server id 3  end_log_pos 202835298     Query   thread_id=36785680      exec_time=0     error_code=0
          SET TIMESTAMP=1438160388/*!*/;
          insert into `queueentry`(`identry`,`idqueue`,`entrydate`,`removedate`,`info`,`prio`,`reserved`,`ObjectContext`,`OriginHost`,`ReservingHost`,`UnreserveDate`,`ReservationCounter`,`OptimisticLockField`)values('
          semtest1233_0',249,'2015-07-29 10:59:30.585','2015-07-29 11:04:30.582','',0,null,'semaphore','',null,null,0,0)
          /*!*/;
          # at 202835298
          #150729 10:59:48 server id 3  end_log_pos 202835325     Xid = 8873293912
          COMMIT/*!*/;
          # at 202835325
          #150729 10:59:48 server id 3  end_log_pos 202835365     GTID 0-3-616369406 cid=8873293914
          /*!100001 SET @@session.gtid_seq_no=616369406*//*!*/;
          BEGIN
          /*!*/;
          # at 202835365
          #150729 10:59:48 server id 3  end_log_pos 202835517     Query   thread_id=36785679      exec_time=0     error_code=0
          SET TIMESTAMP=1438160388/*!*/;
          delete from `queueentry` where ((`OID` = 19599030) and (`OptimisticLockField` = 0))
          /*!*/;
          # at 202835517
          #150729 10:59:48 server id 3  end_log_pos 202835544     Xid = 8873293911
          COMMIT/*!*/;
          # at 202835544
          #150729 10:59:48 server id 3  end_log_pos 202835584     GTID 0-3-616369407 cid=8873293940
          /*!100001 SET @@session.gtid_seq_no=616369407*//*!*/;
          BEGIN
          /*!*/;
          # at 202835584
          #150729 10:59:48 server id 3  end_log_pos 202835736     Query   thread_id=36785680      exec_time=0     error_code=0
          SET TIMESTAMP=1438160388/*!*/;
          delete from `queueentry` where ((`OID` = 19599033) and (`OptimisticLockField` = 0))
          /*!*/;
          # at 202835736
          #150729 10:59:48 server id 3  end_log_pos 202835763     Xid = 8873293937
          COMMIT/*!*/;
          # at 202835763
          #150729 10:59:48 server id 3  end_log_pos 202835803     GTID 0-3-616369408 cid=8873293940
          /*!100001 SET @@session.gtid_seq_no=616369408*//*!*/;
          BEGIN
          /*!*/;
          # at 202835803
          #150729 10:59:48 server id 3  end_log_pos 202835831     Intvar
          SET INSERT_ID=19599035/*!*/;
          # at 202835831
          #150729 10:59:48 server id 3  end_log_pos 202836222     Query   thread_id=36785679      exec_time=0     error_code=0
          SET TIMESTAMP=1438160388/*!*/;
          insert into `queueentry`(`identry`,`idqueue`,`entrydate`,`removedate`,`info`,`prio`,`reserved`,`ObjectContext`,`OriginHost`,`ReservingHost`,`UnreserveDate`,`ReservationCounter`,`OptimisticLockField`)values('semtest123456789_0',249,'2015-07-29 10:59:30.599','2015-07-29 11:04:30.598','',0,null,'semaphore','',null,null,0,0)
          /*!*/;
          # at 202836222
          #150729 10:59:48 server id 3  end_log_pos 202836249     Xid = 8873293938
          COMMIT/*!*/;
          # at 202836249

          And a third case:

          MDEV8302: Deadlock kill GTID 0-3-616369749 (subid 617152583 in_commit=0) due to blocking GTID 0-3-616369748 (subid 617152582 in_commit=0)
          MDEV8302: InnoDB deadlock kill trx_id=11015096074 lock type=32 mode=3
          TRANSACTION 11015096074, ACTIVE 0 sec rollback
          ROLLING BACK 3 lock struct(s), heap size 1184, 2 row lock(s)
          MySQL thread id 321, OS thread handle 0x7f9424dda700, query id 457003 closing tables
          RECORD LOCKS space id 4236271 page no 2353 n bits 344 index `identry_idqueue` of table `optovision`.`queueentry` trx table locks 1 total table locks 2  trx id 11015096074 lock_mode X locks rec but not gap lock hold time 0 wait time before grant 0
          150729 11:06:19 [ERROR] Slave SQL: Error 'Duplicate entry 'semtest123456789_0-249' for key 'identry_idqueue'' on query. Default database: 'optovision'. Query: 'insert into `queueentry`(`identry`,`idqueue`,`entrydate`,`removedate`,`info`,`prio`,`reserved`,`ObjectContext`,`OriginHost`,`ReservingHost`,`UnreserveDate`,`ReservationCounter`,`OptimisticLockField`)values('semtest123456789_0',249,'2015-07-29 10:59:32.582','2015-07-29 11:04:32.581','',0,null,'semaphore','',null,null,0,0)', Gtid 0-3-616369750, Internal MariaDB error code: 1062
          150729 11:06:19 [Warning] Slave: Duplicate entry 'semtest123456789_0-249' for key 'identry_idqueue' Error_code: 1062
          150729 11:06:19 [ERROR] Error running query, slave SQL thread aborted. Fix the problem, and restart the slave SQL thread with "SLAVE START". We stopped at log 'mysql-bin.007999' position 202969179; GTID position '77-3-552,0-3-616369747,3-3-2,2-3-733822,17-3-180,1-3-7'
          MDEV8302: Got deadlock kill in GTID 0-3-616369749 (subid 617152583 in_commit=1)
          MDEV8302: Retry #1 of GTID 0-3-616369749
          MDEV8302: unmark_start_commit(GTID 0-3-616369749) count_committing=342 next->wait_count=342
          MDEV8302:   GTID 0-3-616369749 unmark after signalling next, count_committing=342 next->wait_count=342 next->installed=1
          150729 11:06:19 [ERROR] Slave (additional info): Commit failed due to failure of an earlier commit on which this one depends Error_code: 1964
          150729 11:06:19 [Warning] Slave: Commit failed due to failure of an earlier commit on which this one depends Error_code: 1964
          150729 11:06:19 [ERROR] Error running query, slave SQL thread aborted. Fix the problem, and restart the slave SQL thread with "SLAVE START". We stopped at log 'mysql-bin.007999' position 202969879; GTID position '77-3-552,0-3-616369749,3-3-2,2-3-733822,17-3-180,1-3-7'
          150729 11:06:19 [ERROR] Slave (additional info): Commit failed due to failure of an earlier commit on which this one depends Error_code: 1964
          150729 11:06:19 [Warning] Slave: Commit failed due to failure of an earlier commit on which this one depends Error_code: 1964
          150729 11:06:19 [ERROR] Error running query, slave SQL thread aborted. Fix the problem, and restart the slave SQL thread with "SLAVE START". We stopped at log 'mysql-bin.007999' position 202969879; GTID position '77-3-552,0-3-616369749,3-3-2,2-3-733822,17-3-180,1-3-7'
          150729 11:06:19 [ERROR] Slave (additional info): Commit failed due to failure of an earlier commit on which this one depends Error_code: 1964
          150729 11:06:19 [Warning] Slave: Commit failed due to failure of an earlier commit on which this one depends Error_code: 1964
          150729 11:06:19 [ERROR] Error running query, slave SQL thread aborted. Fix the problem, and restart the slave SQL thread with "SLAVE START". We stopped at log 'mysql-bin.007999' position 202969879; GTID position '77-3-552,0-3-616369749,3-3-2,2-3-733822,17-3-180,1-3-7'
          150729 11:06:19 [ERROR] Slave (additional info): Commit failed due to failure of an earlier commit on which this one depends Error_code: 1964
          150729 11:06:19 [Warning] Slave: Duplicate entry 'semtest123456789_0-249' for key 'identry_idqueue' Error_code: 1062
          150729 11:06:19 [Warning] Slave: Commit failed due to failure of an earlier commit on which this one depends Error_code: 1964
          150729 11:06:19 [ERROR] Error running query, slave SQL thread aborted. Fix the problem, and restart the slave SQL thread with "SLAVE START". We stopped at log 'mysql-bin.007999' position 202969879; GTID position '77-3-552,0-3-616369749,3-3-2,2-3-733822,17-3-180,1-3-7'
          150729 11:06:19 [Note] Slave SQL thread exiting, replication stopped in log 'mysql-bin.007999' at position 202969879; GTID position '77-3-552,0-3-616369749,3-3-2,2-3-733822,17-3-180,1-3-7'

          The binlog:

          #150729 10:59:50 server id 3  end_log_pos 202969219     GTID 0-3-616369748 cid=8873298612
          /*!100001 SET @@session.gtid_seq_no=616369748*//*!*/;
          BEGIN
          /*!*/;
          # at 202969219
          #150729 10:59:50 server id 3  end_log_pos 202969247     Intvar
          SET INSERT_ID=19599380/*!*/;
          # at 202969247
          #150729 10:59:50 server id 3  end_log_pos 202969633     Query   thread_id=36785671      exec_time=0     error_code=0
          SET TIMESTAMP=1438160390/*!*/;
          insert into `queueentry`(`identry`,`idqueue`,`entrydate`,`removedate`,`info`,`prio`,`reserved`,`ObjectContext`,`OriginHost`,`ReservingHost`,`UnreserveDate`,`ReservationCounter`,`OptimisticLockField`)values('
          semtest1233_0',249,'2015-07-29 10:59:32.568','2015-07-29 11:04:32.567','',0,null,'semaphore','',null,null,0,0)
          /*!*/;
          # at 202969633
          #150729 10:59:50 server id 3  end_log_pos 202969660     Xid = 8873298609
          COMMIT/*!*/;
          # at 202969660
          #150729 10:59:50 server id 3  end_log_pos 202969700     GTID 0-3-616369749 cid=8873298612
          /*!100001 SET @@session.gtid_seq_no=616369749*//*!*/;
          BEGIN
          /*!*/;
          # at 202969700
          #150729 10:59:50 server id 3  end_log_pos 202969852     Query   thread_id=36785679      exec_time=0     error_code=0
          SET TIMESTAMP=1438160390/*!*/;
          delete from `queueentry` where ((`OID` = 19599377) and (`OptimisticLockField` = 0))
          /*!*/;
          # at 202969852
          #150729 10:59:50 server id 3  end_log_pos 202969879     Xid = 8873298611
          COMMIT/*!*/;
          # at 202969879
          #150729 10:59:50 server id 3  end_log_pos 202969919     GTID 0-3-616369750 cid=8873298654
          /*!100001 SET @@session.gtid_seq_no=616369750*//*!*/;
          BEGIN
          /*!*/;
          # at 202969919
          #150729 10:59:50 server id 3  end_log_pos 202969947     Intvar
          SET INSERT_ID=19599382/*!*/;
          # at 202969947
          #150729 10:59:50 server id 3  end_log_pos 202970338     Query   thread_id=36785678      exec_time=0     error_code=0
          SET TIMESTAMP=1438160390/*!*/;
          insert into `queueentry`(`identry`,`idqueue`,`entrydate`,`removedate`,`info`,`prio`,`reserved`,`ObjectContext`,`OriginHost`,`ReservingHost`,`UnreserveDate`,`ReservationCounter`,`OptimisticLockField`)values('semtest123456789_0',249,'2015-07-29 10:59:32.582','2015-07-29 11:04:32.581','',0,null,'semaphore','',null,null,0,0)
          /*!*/;
          # at 202970338
          #150729 10:59:50 server id 3  end_log_pos 202970365     Xid = 8873298650
          COMMIT/*!*/;
          # at 202970365
          #150729 10:59:50 server id 3  end_log_pos 202970405     GTID 0-3-616369751 cid=8873298654
          /*!100001 SET @@session.gtid_seq_no=616369751*//*!*/;
          BEGIN
          /*!*/;
          # at 202970405
          #150729 10:59:50 server id 3  end_log_pos 202970557     Query   thread_id=36785671      exec_time=0     error_code=0
          SET TIMESTAMP=1438160390/*!*/;
          delete from `queueentry` where ((`OID` = 19599380) and (`OptimisticLockField` = 0))
          /*!*/;
          # at 202970557
          #150729 10:59:50 server id 3  end_log_pos 202970584     Xid = 8873298665
          COMMIT/*!*/;
          # at 202970584

          In this case, a third transaction was part of the second group, consisting of quite a few statements, but all for a different table.

          Hope, this helps to track things down. If you need more cases or another test...

          thomas.mischke Thomas Mischke added a comment - Here we go. I compiled the new version, added the innodb_print_all_deadlocks and reproduced the problem. Here is the output of the log file. Just for completeness: There are much more messages about MDEV8302 and deadlocks, but they all belong to transactions long before the deadlock, so I do not report them. MDEV8302: Deadlock kill GTID 0-3-616241346 (subid 617021915 in_commit=0) due to blocking GTID 0-3-616241345 (subid 617021914 in_commit=0) MDEV8302: mark_start_commit(GTID 0-3-616241346) after killed MDEV8302: InnoDB deadlock kill trx_id=11014855138 lock type=32 mode=3 TRANSACTION 11014855138, ACTIVE 0 sec rollback ROLLING BACK 3 lock struct(s), heap size 1184, 2 row lock(s) MySQL thread id 36, OS thread handle 0x7f9424d91700, query id 88346 closing tables RECORD LOCKS space id 4236271 page no 2329 n bits 440 index `identry_idqueue` of table `optovision`.`queueentry` trx table locks 1 total table locks 2 trx id 11014855138 lock_mode X locks rec but not gap lock hold time 0 wait time before grant 0 MDEV8302: Got deadlock kill in GTID 0-3-616241346 (subid 617021915 in_commit=1) MDEV8302: Retry #1 of GTID 0-3-616241346 150729 10:36:50 [ERROR] Slave SQL: Error 'Duplicate entry 'semtest123456789_0-249' for key 'identry_idqueue'' on query. Default database: 'optovision'. Query: 'insert into `queueentry`(`identry`,`idqueue`,`entrydate`,`removedate`,`info`,`prio`,`reserved`,`ObjectContext`,`OriginHost`,`ReservingHost`,`UnreserveDate`,`ReservationCounter`,`OptimisticLockField`)values('semtest123456789_0',249,'2015-07-29 10:36:24.067','2015-07-29 10:41:24.066','',0,null,'semaphore','',null,null,0,0)', Gtid 0-3-616241347, Internal MariaDB error code: 1062 MDEV8302: unmark_start_commit(GTID 0-3-616241346) count_committing=4199 next->wait_count=4199 MDEV8302: GTID 0-3-616241346 unmark after signalling next, count_committing=4199 next->wait_count=4199 next->installed=1 150729 10:36:50 [Warning] Slave: Duplicate entry 'semtest123456789_0-249' for key 'identry_idqueue' Error_code: 1062 150729 10:36:50 [ERROR] Error running query, slave SQL thread aborted. Fix the problem, and restart the slave SQL thread with "SLAVE START". We stopped at log 'mysql-bin.007999' position 50282813; GTID position '77-3-552,0-3-616241344,3-3-2,2-3-733822,17-3-180,1-3-7' 150729 10:36:50 [ERROR] Slave (additional info): Commit failed due to failure of an earlier commit on which this one depends Error_code: 1964 150729 10:36:50 [Warning] Slave: Commit failed due to failure of an earlier commit on which this one depends Error_code: 1964 150729 10:36:50 [ERROR] Error running query, slave SQL thread aborted. Fix the problem, and restart the slave SQL thread with "SLAVE START". We stopped at log 'mysql-bin.007999' position 50283513; GTID position '77-3-552,0-3-616241346,3-3-2,2-3-733822,17-3-180,1-3-7' 150729 10:36:50 [ERROR] Slave (additional info): Commit failed due to failure of an earlier commit on which this one depends Error_code: 1964 150729 10:36:50 [Warning] Slave: Commit failed due to failure of an earlier commit on which this one depends Error_code: 1964 150729 10:36:50 [ERROR] Error running query, slave SQL thread aborted. Fix the problem, and restart the slave SQL thread with "SLAVE START". We stopped at log 'mysql-bin.007999' position 50283513; GTID position '77-3-552,0-3-616241346,3-3-2,2-3-733822,17-3-180,1-3-7' 150729 10:36:50 [ERROR] Slave (additional info): Commit failed due to failure of an earlier commit on which this one depends Error_code: 1964 150729 10:36:50 [Warning] Slave: Commit failed due to failure of an earlier commit on which this one depends Error_code: 1964 150729 10:36:50 [ERROR] Error running query, slave SQL thread aborted. Fix the problem, and restart the slave SQL thread with "SLAVE START". We stopped at log 'mysql-bin.007999' position 50283513; GTID position '77-3-552,0-3-616241346,3-3-2,2-3-733822,17-3-180,1-3-7' 150729 10:36:50 [ERROR] Slave (additional info): Commit failed due to failure of an earlier commit on which this one depends Error_code: 1964 150729 10:36:50 [Warning] Slave: Duplicate entry 'semtest123456789_0-249' for key 'identry_idqueue' Error_code: 1062 150729 10:36:50 [Warning] Slave: Commit failed due to failure of an earlier commit on which this one depends Error_code: 1964 150729 10:36:50 [ERROR] Error running query, slave SQL thread aborted. Fix the problem, and restart the slave SQL thread with "SLAVE START". We stopped at log 'mysql-bin.007999' position 50283513; GTID position '77-3-552,0-3-616241346,3-3-2,2-3-733822,17-3-180,1-3-7' 150729 10:36:50 [Note] Slave SQL thread exiting, replication stopped in log 'mysql-bin.007999' at position 50283513; GTID position '77-3-552,0-3-616241346,3-3-2,2-3-733822,17-3-180,1-3-7' The binlog of the master looks as folows (again I made sure that the delete of the first group deletes the key from the insert in the second group; further I checked that no MDEV8302 or deadlocks were reported when this key was written first): #150729 10:36:41 server id 3 end_log_pos 50282853 GTID 0-3-616241345 cid=8871685554 /*!100001 SET @@session.gtid_seq_no=616241345*//*!*/; BEGIN /*!*/; # at 50282853 #150729 10:36:41 server id 3 end_log_pos 50282881 Intvar SET INSERT_ID=19563752/*!*/; # at 50282881 #150729 10:36:41 server id 3 end_log_pos 50283267 Query thread_id=36782520 exec_time=0 error_code=0 SET TIMESTAMP=1438159001/*!*/; insert into `queueentry`(`identry`,`idqueue`,`entrydate`,`removedate`,`info`,`prio`,`reserved`,`ObjectContext`,`OriginHost`,`ReservingHost`,`UnreserveDate`,`ReservationCounter`,`OptimisticLockField`)values(' semtest1233_0',249,'2015-07-29 10:36:24.053','2015-07-29 10:41:24.052','',0,null,'semaphore','',null,null,0,0) /*!*/; # at 50283267 #150729 10:36:41 server id 3 end_log_pos 50283294 Xid = 8871685551 COMMIT/*!*/; # at 50283294 #150729 10:36:41 server id 3 end_log_pos 50283334 GTID 0-3-616241346 cid=8871685554 /*!100001 SET @@session.gtid_seq_no=616241346*//*!*/; BEGIN /*!*/; # at 50283334 #150729 10:36:41 server id 3 end_log_pos 50283486 Query thread_id=36782495 exec_time=0 error_code=0 SET TIMESTAMP=1438159001/*!*/; delete from `queueentry` where ((`OID` = 19563749) and (`OptimisticLockField` = 0)) /*!*/; # at 50283486 #150729 10:36:41 server id 3 end_log_pos 50283513 Xid = 8871685553 COMMIT/*!*/; # at 50283513 #150729 10:36:41 server id 3 end_log_pos 50283553 GTID 0-3-616241347 cid=8871685582 /*!100001 SET @@session.gtid_seq_no=616241347*//*!*/; BEGIN /*!*/; # at 50283553 #150729 10:36:41 server id 3 end_log_pos 50283581 Intvar SET INSERT_ID=19563754/*!*/; # at 50283581 #150729 10:36:41 server id 3 end_log_pos 50283972 Query thread_id=36782519 exec_time=0 error_code=0 SET TIMESTAMP=1438159001/*!*/; insert into `queueentry`(`identry`,`idqueue`,`entrydate`,`removedate`,`info`,`prio`,`reserved`,`ObjectContext`,`OriginHost`,`ReservingHost`,`UnreserveDate`,`ReservationCounter`,`OptimisticLockField`)values('semtest123456789_0',249,'2015-07-29 10:36:24.067','2015-07-29 10:41:24.066','',0,null,'semaphore','',null,null,0,0) /*!*/; # at 50283972 #150729 10:36:41 server id 3 end_log_pos 50283999 Xid = 8871685578 COMMIT/*!*/; # at 50283999 #150729 10:36:41 server id 3 end_log_pos 50284039 GTID 0-3-616241348 cid=8871685582 /*!100001 SET @@session.gtid_seq_no=616241348*//*!*/; BEGIN /*!*/; # at 50284039 #150729 10:36:41 server id 3 end_log_pos 50284191 Query thread_id=36782520 exec_time=0 error_code=0 SET TIMESTAMP=1438159001/*!*/; delete from `queueentry` where ((`OID` = 19563752) and (`OptimisticLockField` = 0)) /*!*/; # at 50284191 #150729 10:36:41 server id 3 end_log_pos 50284218 Xid = 8871685585 COMMIT/*!*/; # at 50284218 Here is a second case, this time without mark_start_commit: The log file: MDEV8302: Deadlock kill GTID 0-3-616369406 (subid 617151562 in_commit=0) due to blocking GTID 0-3-616369405 (subid 617151561 in_commit=0) MDEV8302: InnoDB deadlock kill trx_id=11015095365 lock type=32 mode=3 TRANSACTION 11015095365, ACTIVE 0 sec rollback ROLLING BACK 3 lock struct(s), heap size 1184, 2 row lock(s) MySQL thread id 270, OS thread handle 0x7f918e119700, query id 455867 closing tables RECORD LOCKS space id 4236271 page no 2353 n bits 312 index `identry_idqueue` of table `optovision`.`queueentry` trx table locks 1 total table locks 2 trx id 11015095365 lock_mode X locks rec but not gap lock hold time 0 wait time before grant 0 MDEV8302: Got deadlock kill in GTID 0-3-616369406 (subid 617151562 in_commit=1) MDEV8302: Retry #1 of GTID 0-3-616369406 MDEV8302: unmark_start_commit(GTID 0-3-616369406) count_committing=127 next->wait_count=127 MDEV8302: GTID 0-3-616369406 unmark after signalling next, count_committing=127 next->wait_count=127 next->installed=1 150729 11:00:23 [ERROR] Slave SQL: Error 'Duplicate entry 'semtest123456789_0-249' for key 'identry_idqueue'' on query. Default database: 'optovision'. Query: 'insert into `queueentry`(`identry`,`idqueue`,`entrydate`,`removedate`,`info`,`prio`,`reserved`,`ObjectContext`,`OriginHost`,`ReservingHost`,`UnreserveDate`,`ReservationCounter`,`OptimisticLockField`)values('semtest123456789_0',249,'2015-07-29 10:59:30.599','2015-07-29 11:04:30.598','',0,null,'semaphore','',null,null,0,0)', Gtid 0-3-616369408, Internal MariaDB error code: 1062 150729 11:00:23 [Warning] Slave: Duplicate entry 'semtest123456789_0-249' for key 'identry_idqueue' Error_code: 1062 150729 11:00:23 [ERROR] Error running query, slave SQL thread aborted. Fix the problem, and restart the slave SQL thread with "SLAVE START". We stopped at log 'mysql-bin.007999' position 202834844; GTID position '77-3-552,0-3-616369404,3-3-2,2-3-733822,17-3-180,1-3-7' 150729 11:00:23 [Note] Slave SQL thread exiting, replication stopped in log 'mysql-bin.007999' at position 202835763; GTID position '77-3-552,0-3-616369407,3-3-2,2-3-733822,17-3-180,1-3-7' And the binlog of the master: #150729 10:59:48 server id 3 end_log_pos 202834884 GTID 0-3-616369405 cid=8873293914 /*!100001 SET @@session.gtid_seq_no=616369405*//*!*/; BEGIN /*!*/; # at 202834884 #150729 10:59:48 server id 3 end_log_pos 202834912 Intvar SET INSERT_ID=19599033/*!*/; # at 202834912 #150729 10:59:48 server id 3 end_log_pos 202835298 Query thread_id=36785680 exec_time=0 error_code=0 SET TIMESTAMP=1438160388/*!*/; insert into `queueentry`(`identry`,`idqueue`,`entrydate`,`removedate`,`info`,`prio`,`reserved`,`ObjectContext`,`OriginHost`,`ReservingHost`,`UnreserveDate`,`ReservationCounter`,`OptimisticLockField`)values(' semtest1233_0',249,'2015-07-29 10:59:30.585','2015-07-29 11:04:30.582','',0,null,'semaphore','',null,null,0,0) /*!*/; # at 202835298 #150729 10:59:48 server id 3 end_log_pos 202835325 Xid = 8873293912 COMMIT/*!*/; # at 202835325 #150729 10:59:48 server id 3 end_log_pos 202835365 GTID 0-3-616369406 cid=8873293914 /*!100001 SET @@session.gtid_seq_no=616369406*//*!*/; BEGIN /*!*/; # at 202835365 #150729 10:59:48 server id 3 end_log_pos 202835517 Query thread_id=36785679 exec_time=0 error_code=0 SET TIMESTAMP=1438160388/*!*/; delete from `queueentry` where ((`OID` = 19599030) and (`OptimisticLockField` = 0)) /*!*/; # at 202835517 #150729 10:59:48 server id 3 end_log_pos 202835544 Xid = 8873293911 COMMIT/*!*/; # at 202835544 #150729 10:59:48 server id 3 end_log_pos 202835584 GTID 0-3-616369407 cid=8873293940 /*!100001 SET @@session.gtid_seq_no=616369407*//*!*/; BEGIN /*!*/; # at 202835584 #150729 10:59:48 server id 3 end_log_pos 202835736 Query thread_id=36785680 exec_time=0 error_code=0 SET TIMESTAMP=1438160388/*!*/; delete from `queueentry` where ((`OID` = 19599033) and (`OptimisticLockField` = 0)) /*!*/; # at 202835736 #150729 10:59:48 server id 3 end_log_pos 202835763 Xid = 8873293937 COMMIT/*!*/; # at 202835763 #150729 10:59:48 server id 3 end_log_pos 202835803 GTID 0-3-616369408 cid=8873293940 /*!100001 SET @@session.gtid_seq_no=616369408*//*!*/; BEGIN /*!*/; # at 202835803 #150729 10:59:48 server id 3 end_log_pos 202835831 Intvar SET INSERT_ID=19599035/*!*/; # at 202835831 #150729 10:59:48 server id 3 end_log_pos 202836222 Query thread_id=36785679 exec_time=0 error_code=0 SET TIMESTAMP=1438160388/*!*/; insert into `queueentry`(`identry`,`idqueue`,`entrydate`,`removedate`,`info`,`prio`,`reserved`,`ObjectContext`,`OriginHost`,`ReservingHost`,`UnreserveDate`,`ReservationCounter`,`OptimisticLockField`)values('semtest123456789_0',249,'2015-07-29 10:59:30.599','2015-07-29 11:04:30.598','',0,null,'semaphore','',null,null,0,0) /*!*/; # at 202836222 #150729 10:59:48 server id 3 end_log_pos 202836249 Xid = 8873293938 COMMIT/*!*/; # at 202836249 And a third case: MDEV8302: Deadlock kill GTID 0-3-616369749 (subid 617152583 in_commit=0) due to blocking GTID 0-3-616369748 (subid 617152582 in_commit=0) MDEV8302: InnoDB deadlock kill trx_id=11015096074 lock type=32 mode=3 TRANSACTION 11015096074, ACTIVE 0 sec rollback ROLLING BACK 3 lock struct(s), heap size 1184, 2 row lock(s) MySQL thread id 321, OS thread handle 0x7f9424dda700, query id 457003 closing tables RECORD LOCKS space id 4236271 page no 2353 n bits 344 index `identry_idqueue` of table `optovision`.`queueentry` trx table locks 1 total table locks 2 trx id 11015096074 lock_mode X locks rec but not gap lock hold time 0 wait time before grant 0 150729 11:06:19 [ERROR] Slave SQL: Error 'Duplicate entry 'semtest123456789_0-249' for key 'identry_idqueue'' on query. Default database: 'optovision'. Query: 'insert into `queueentry`(`identry`,`idqueue`,`entrydate`,`removedate`,`info`,`prio`,`reserved`,`ObjectContext`,`OriginHost`,`ReservingHost`,`UnreserveDate`,`ReservationCounter`,`OptimisticLockField`)values('semtest123456789_0',249,'2015-07-29 10:59:32.582','2015-07-29 11:04:32.581','',0,null,'semaphore','',null,null,0,0)', Gtid 0-3-616369750, Internal MariaDB error code: 1062 150729 11:06:19 [Warning] Slave: Duplicate entry 'semtest123456789_0-249' for key 'identry_idqueue' Error_code: 1062 150729 11:06:19 [ERROR] Error running query, slave SQL thread aborted. Fix the problem, and restart the slave SQL thread with "SLAVE START". We stopped at log 'mysql-bin.007999' position 202969179; GTID position '77-3-552,0-3-616369747,3-3-2,2-3-733822,17-3-180,1-3-7' MDEV8302: Got deadlock kill in GTID 0-3-616369749 (subid 617152583 in_commit=1) MDEV8302: Retry #1 of GTID 0-3-616369749 MDEV8302: unmark_start_commit(GTID 0-3-616369749) count_committing=342 next->wait_count=342 MDEV8302: GTID 0-3-616369749 unmark after signalling next, count_committing=342 next->wait_count=342 next->installed=1 150729 11:06:19 [ERROR] Slave (additional info): Commit failed due to failure of an earlier commit on which this one depends Error_code: 1964 150729 11:06:19 [Warning] Slave: Commit failed due to failure of an earlier commit on which this one depends Error_code: 1964 150729 11:06:19 [ERROR] Error running query, slave SQL thread aborted. Fix the problem, and restart the slave SQL thread with "SLAVE START". We stopped at log 'mysql-bin.007999' position 202969879; GTID position '77-3-552,0-3-616369749,3-3-2,2-3-733822,17-3-180,1-3-7' 150729 11:06:19 [ERROR] Slave (additional info): Commit failed due to failure of an earlier commit on which this one depends Error_code: 1964 150729 11:06:19 [Warning] Slave: Commit failed due to failure of an earlier commit on which this one depends Error_code: 1964 150729 11:06:19 [ERROR] Error running query, slave SQL thread aborted. Fix the problem, and restart the slave SQL thread with "SLAVE START". We stopped at log 'mysql-bin.007999' position 202969879; GTID position '77-3-552,0-3-616369749,3-3-2,2-3-733822,17-3-180,1-3-7' 150729 11:06:19 [ERROR] Slave (additional info): Commit failed due to failure of an earlier commit on which this one depends Error_code: 1964 150729 11:06:19 [Warning] Slave: Commit failed due to failure of an earlier commit on which this one depends Error_code: 1964 150729 11:06:19 [ERROR] Error running query, slave SQL thread aborted. Fix the problem, and restart the slave SQL thread with "SLAVE START". We stopped at log 'mysql-bin.007999' position 202969879; GTID position '77-3-552,0-3-616369749,3-3-2,2-3-733822,17-3-180,1-3-7' 150729 11:06:19 [ERROR] Slave (additional info): Commit failed due to failure of an earlier commit on which this one depends Error_code: 1964 150729 11:06:19 [Warning] Slave: Duplicate entry 'semtest123456789_0-249' for key 'identry_idqueue' Error_code: 1062 150729 11:06:19 [Warning] Slave: Commit failed due to failure of an earlier commit on which this one depends Error_code: 1964 150729 11:06:19 [ERROR] Error running query, slave SQL thread aborted. Fix the problem, and restart the slave SQL thread with "SLAVE START". We stopped at log 'mysql-bin.007999' position 202969879; GTID position '77-3-552,0-3-616369749,3-3-2,2-3-733822,17-3-180,1-3-7' 150729 11:06:19 [Note] Slave SQL thread exiting, replication stopped in log 'mysql-bin.007999' at position 202969879; GTID position '77-3-552,0-3-616369749,3-3-2,2-3-733822,17-3-180,1-3-7' The binlog: #150729 10:59:50 server id 3 end_log_pos 202969219 GTID 0-3-616369748 cid=8873298612 /*!100001 SET @@session.gtid_seq_no=616369748*//*!*/; BEGIN /*!*/; # at 202969219 #150729 10:59:50 server id 3 end_log_pos 202969247 Intvar SET INSERT_ID=19599380/*!*/; # at 202969247 #150729 10:59:50 server id 3 end_log_pos 202969633 Query thread_id=36785671 exec_time=0 error_code=0 SET TIMESTAMP=1438160390/*!*/; insert into `queueentry`(`identry`,`idqueue`,`entrydate`,`removedate`,`info`,`prio`,`reserved`,`ObjectContext`,`OriginHost`,`ReservingHost`,`UnreserveDate`,`ReservationCounter`,`OptimisticLockField`)values(' semtest1233_0',249,'2015-07-29 10:59:32.568','2015-07-29 11:04:32.567','',0,null,'semaphore','',null,null,0,0) /*!*/; # at 202969633 #150729 10:59:50 server id 3 end_log_pos 202969660 Xid = 8873298609 COMMIT/*!*/; # at 202969660 #150729 10:59:50 server id 3 end_log_pos 202969700 GTID 0-3-616369749 cid=8873298612 /*!100001 SET @@session.gtid_seq_no=616369749*//*!*/; BEGIN /*!*/; # at 202969700 #150729 10:59:50 server id 3 end_log_pos 202969852 Query thread_id=36785679 exec_time=0 error_code=0 SET TIMESTAMP=1438160390/*!*/; delete from `queueentry` where ((`OID` = 19599377) and (`OptimisticLockField` = 0)) /*!*/; # at 202969852 #150729 10:59:50 server id 3 end_log_pos 202969879 Xid = 8873298611 COMMIT/*!*/; # at 202969879 #150729 10:59:50 server id 3 end_log_pos 202969919 GTID 0-3-616369750 cid=8873298654 /*!100001 SET @@session.gtid_seq_no=616369750*//*!*/; BEGIN /*!*/; # at 202969919 #150729 10:59:50 server id 3 end_log_pos 202969947 Intvar SET INSERT_ID=19599382/*!*/; # at 202969947 #150729 10:59:50 server id 3 end_log_pos 202970338 Query thread_id=36785678 exec_time=0 error_code=0 SET TIMESTAMP=1438160390/*!*/; insert into `queueentry`(`identry`,`idqueue`,`entrydate`,`removedate`,`info`,`prio`,`reserved`,`ObjectContext`,`OriginHost`,`ReservingHost`,`UnreserveDate`,`ReservationCounter`,`OptimisticLockField`)values('semtest123456789_0',249,'2015-07-29 10:59:32.582','2015-07-29 11:04:32.581','',0,null,'semaphore','',null,null,0,0) /*!*/; # at 202970338 #150729 10:59:50 server id 3 end_log_pos 202970365 Xid = 8873298650 COMMIT/*!*/; # at 202970365 #150729 10:59:50 server id 3 end_log_pos 202970405 GTID 0-3-616369751 cid=8873298654 /*!100001 SET @@session.gtid_seq_no=616369751*//*!*/; BEGIN /*!*/; # at 202970405 #150729 10:59:50 server id 3 end_log_pos 202970557 Query thread_id=36785671 exec_time=0 error_code=0 SET TIMESTAMP=1438160390/*!*/; delete from `queueentry` where ((`OID` = 19599380) and (`OptimisticLockField` = 0)) /*!*/; # at 202970557 #150729 10:59:50 server id 3 end_log_pos 202970584 Xid = 8873298665 COMMIT/*!*/; # at 202970584 In this case, a third transaction was part of the second group, consisting of quite a few statements, but all for a different table. Hope, this helps to track things down. If you need more cases or another test...

          Thanks a lot, Thomas!

          I need a bit of time to fully analyse the new information, but there is
          something interesting in there:

          TRANSACTION 11015096074, ACTIVE 0 sec rollback

          If the transaction T2 that we are killing is already rolling back, then that
          could explain the problem. T1 would be able to proceed before T2 can handle
          the kill, and we could start following transactions too early.

          So I need to look at the code to see if T2 is really rolling back, and if so
          why. I will get back to you when I know more, and if I need any additional
          tests.

          Thanks,

          • Kristian.
          knielsen Kristian Nielsen added a comment - Thanks a lot, Thomas! I need a bit of time to fully analyse the new information, but there is something interesting in there: TRANSACTION 11015096074, ACTIVE 0 sec rollback If the transaction T2 that we are killing is already rolling back, then that could explain the problem. T1 would be able to proceed before T2 can handle the kill, and we could start following transactions too early. So I need to look at the code to see if T2 is really rolling back, and if so why. I will get back to you when I know more, and if I need any additional tests. Thanks, Kristian.
          elenst Elena Stepanova made changes -
          Labels need_feedback parallelslave replication parallelslave replication

          Ok, I think I (finally) got it. I am attaching a patch that should fix the
          bug (it goes on top of the previous patch).

          This looks like another variant of MDEV-7458. There were two places in the
          code where transaction T2 could rollback without first informing following
          transactions about this fact. This created a rare race, where following T3
          could be allowed to start too early, before T2 had completed its transaction
          retry. This could lead to the duplicate key errors seen in this bug.

          I think that there is a good chance that the patch fixes the bug; at least,
          it fixes a real bug, and the symptoms match the reported behaviour. But it
          is impossible to be sure, so it would be great if you could try running your
          test slave with this new patch, and verify that the duplicate key error is
          now gone.

          You should still get a bunch of printouts in the error log, but that is
          fine, the transaction conflicts are not a problem in themselves. But the
          duplicate key error that causes replication to stop should be gone,
          hopefully.

          Once again thanks for all your efforts in helping tracking this one down, it
          was a hard one for sure.

          • Kristian.
          knielsen Kristian Nielsen added a comment - Ok, I think I (finally) got it. I am attaching a patch that should fix the bug (it goes on top of the previous patch). This looks like another variant of MDEV-7458 . There were two places in the code where transaction T2 could rollback without first informing following transactions about this fact. This created a rare race, where following T3 could be allowed to start too early, before T2 had completed its transaction retry. This could lead to the duplicate key errors seen in this bug. I think that there is a good chance that the patch fixes the bug; at least, it fixes a real bug, and the symptoms match the reported behaviour. But it is impossible to be sure, so it would be great if you could try running your test slave with this new patch, and verify that the duplicate key error is now gone. You should still get a bunch of printouts in the error log, but that is fine, the transaction conflicts are not a problem in themselves. But the duplicate key error that causes replication to stop should be gone, hopefully. Once again thanks for all your efforts in helping tracking this one down, it was a hard one for sure. Kristian.
          knielsen Kristian Nielsen made changes -
          Attachment mdev8302.patch [ 39001 ]
          knielsen Kristian Nielsen made changes -
          Fix Version/s 10.0.21 [ 19406 ]
          Fix Version/s 10.0 [ 16000 ]
          knielsen Kristian Nielsen made changes -
          Attachment mdev8302-release.patch [ 39002 ]

          For reference, I also attached (mdev8302-release.patch) the patch that I
          have pushed to 10.0.21, without any debug printouts.

          knielsen Kristian Nielsen added a comment - For reference, I also attached (mdev8302-release.patch) the patch that I have pushed to 10.0.21, without any debug printouts.
          knielsen Kristian Nielsen made changes -
          Fix Version/s 10.1.7 [ 19604 ]

          Pushed into 10.0.21 and 10.1.7. Let's keep the bug open to see if Thomas Mischke can verify the fix on his setup.

          knielsen Kristian Nielsen added a comment - Pushed into 10.0.21 and 10.1.7. Let's keep the bug open to see if Thomas Mischke can verify the fix on his setup.

          Somehow I am unable to build MariaDB when the patch is applied. I always get:

          Linking CXX shared library libmysqld.so
          libmysqld.a(handler.cc.o): In function `ha_rollback_trans(THD*, bool)':
          /root/mariadb-10.0.20/sql/handler.cc:1600: undefined reference to `rpl_group_info::unmark_start_commit()'
          collect2: ld returned 1 exit status
          make[2]: *** [libmysqld/libmysqld.so.18] Error 1
          make[1]: *** [libmysqld/CMakeFiles/libmysqld.dir/all] Error 2
          make: *** [all] Error 2

          Could it be that the patch misses something, or is this my fault?

          thomas.mischke Thomas Mischke added a comment - Somehow I am unable to build MariaDB when the patch is applied. I always get: Linking CXX shared library libmysqld.so libmysqld.a(handler.cc.o): In function `ha_rollback_trans(THD*, bool)': /root/mariadb-10.0.20/sql/handler.cc:1600: undefined reference to `rpl_group_info::unmark_start_commit()' collect2: ld returned 1 exit status make[2]: *** [libmysqld/libmysqld.so.18] Error 1 make[1]: *** [libmysqld/CMakeFiles/libmysqld.dir/all] Error 2 make: *** [all] Error 2 Could it be that the patch misses something, or is this my fault?

          Bummer I messed up, it is the libmysqld that does not build

          You can either build with cmake -DWITH_EMBEDDED_SERVER=OFF, or wait a bit and I'll supply (and push) a fixed patch.

          knielsen Kristian Nielsen added a comment - Bummer I messed up, it is the libmysqld that does not build You can either build with cmake -DWITH_EMBEDDED_SERVER=OFF, or wait a bit and I'll supply (and push) a fixed patch.
          knielsen Kristian Nielsen made changes -
          Attachment mdev8302-fix-embedded-server.patch [ 39003 ]

          I have attached additional patch mdev8302-fix-embedded-server.patch, which fixes the build failure.

          knielsen Kristian Nielsen added a comment - I have attached additional patch mdev8302-fix-embedded-server.patch, which fixes the build failure.
          serg Sergei Golubchik made changes -
          Fix Version/s 10.0 [ 16000 ]
          Fix Version/s 10.0.21 [ 19406 ]

          We were trying hard, but we were not able to reproduce the duplicate entry error with this fix. When we switched back to the version without the fix, it takes just seconds to appear, but with the new version it is running now for nearly an hour without any error. So we believe this issue is fixed and we are looking forward to have it in the next release.

          Thanks a lot!

          thomas.mischke Thomas Mischke added a comment - We were trying hard, but we were not able to reproduce the duplicate entry error with this fix. When we switched back to the version without the fix, it takes just seconds to appear, but with the new version it is running now for nearly an hour without any error. So we believe this issue is fixed and we are looking forward to have it in the next release. Thanks a lot!

          Great news. Good to know that this one is finally solved.

          The fix should appear in 10.0.21 and 10.1.7

          http://lists.askmonty.org/pipermail/commits/2015-August/008223.html
          http://lists.askmonty.org/pipermail/commits/2015-August/008230.html

          knielsen Kristian Nielsen added a comment - Great news. Good to know that this one is finally solved. The fix should appear in 10.0.21 and 10.1.7 http://lists.askmonty.org/pipermail/commits/2015-August/008223.html http://lists.askmonty.org/pipermail/commits/2015-August/008230.html
          knielsen Kristian Nielsen made changes -
          Fix Version/s 10.0.21 [ 19406 ]
          Fix Version/s 10.0 [ 16000 ]
          Resolution Fixed [ 1 ]
          Status In Progress [ 3 ] Closed [ 6 ]
          serg Sergei Golubchik made changes -
          Workflow MariaDB v3 [ 69888 ] MariaDB v4 [ 149260 ]

          People

            knielsen Kristian Nielsen
            thomas.mischke Thomas Mischke
            Votes:
            0 Vote for this issue
            Watchers:
            6 Start watching this issue

            Dates

              Created:
              Updated:
              Resolved:

              Git Integration

                Error rendering 'com.xiplink.jira.git.jira_git_plugin:git-issue-webpanel'. Please contact your Jira administrators.