From 0458377f54dee25228ba93b4857970dc1c046917 Mon Sep 17 00:00:00 2001 From: Oleksiy Kovyrin Date: Wed, 9 Nov 2022 16:13:33 -0500 Subject: [PATCH 01/32] Initial WIP implementation of Vitess VStream replicator support Things that work: * The code successfully follows a given keyspace in a Vitess cluster (a specific shard or all shards) and pushes RowMap objects to a given producer. Still need to implement/fix/figure out: * Position store is stubbed out for now and position recovery is not implemented, so it always starts at the current stream position. * I'm not sure I'm handling GRPC stream shutdown properly. * Had to disable OpenCensus dependency for now because they depend on an ancient grpc version that conflicts with the grpc dependency used by vitess. --- pom.xml | 11 +- .../java/com/zendesk/maxwell/Maxwell.java | 99 +++-- .../com/zendesk/maxwell/MaxwellConfig.java | 18 +- .../com/zendesk/maxwell/MaxwellContext.java | 5 +- .../zendesk/maxwell/MaxwellVitessConfig.java | 23 ++ .../maxwell/monitoring/MaxwellMetrics.java | 25 +- .../maxwell/replication/VStreamObserver.java | 69 ++++ .../replication/VStreamReplicator.java | 386 ++++++++++++++++++ .../replication/vitess/ColumnMetaData.java | 41 ++ .../replication/vitess/KeyMetaData.java | 26 ++ .../vitess/ReplicationMessageColumn.java | 69 ++++ .../maxwell/replication/vitess/Vgtid.java | 157 +++++++ .../replication/vitess/VitessColumn.java | 23 ++ .../replication/vitess/VitessColumnValue.java | 76 ++++ .../replication/vitess/VitessSchema.java | 125 ++++++ .../replication/vitess/VitessTable.java | 89 ++++ .../replication/vitess/VitessType.java | 163 ++++++++ .../maxwell/schema/VitessPositionStore.java | 20 + .../zendesk/maxwell/util/AbstractConfig.java | 13 + 19 files changed, 1380 insertions(+), 58 deletions(-) create mode 100644 src/main/java/com/zendesk/maxwell/MaxwellVitessConfig.java create mode 100644 src/main/java/com/zendesk/maxwell/replication/VStreamObserver.java create mode 100644 src/main/java/com/zendesk/maxwell/replication/VStreamReplicator.java create mode 100644 src/main/java/com/zendesk/maxwell/replication/vitess/ColumnMetaData.java create mode 100644 src/main/java/com/zendesk/maxwell/replication/vitess/KeyMetaData.java create mode 100644 src/main/java/com/zendesk/maxwell/replication/vitess/ReplicationMessageColumn.java create mode 100644 src/main/java/com/zendesk/maxwell/replication/vitess/Vgtid.java create mode 100644 src/main/java/com/zendesk/maxwell/replication/vitess/VitessColumn.java create mode 100644 src/main/java/com/zendesk/maxwell/replication/vitess/VitessColumnValue.java create mode 100644 src/main/java/com/zendesk/maxwell/replication/vitess/VitessSchema.java create mode 100644 src/main/java/com/zendesk/maxwell/replication/vitess/VitessTable.java create mode 100644 src/main/java/com/zendesk/maxwell/replication/vitess/VitessType.java create mode 100644 src/main/java/com/zendesk/maxwell/schema/VitessPositionStore.java diff --git a/pom.xml b/pom.xml index a0f6439f5..c586510a7 100644 --- a/pom.xml +++ b/pom.xml @@ -173,6 +173,12 @@ 15.3 + + io.vitess + vitess-grpc-client + 14.0.1 + + org.apache.logging.log4j @@ -261,8 +267,9 @@ simpleclient_servlet 0.9.0 + - + diff --git a/src/main/java/com/zendesk/maxwell/Maxwell.java b/src/main/java/com/zendesk/maxwell/Maxwell.java index 7cc17a2f8..61575ab0e 100644 --- a/src/main/java/com/zendesk/maxwell/Maxwell.java +++ b/src/main/java/com/zendesk/maxwell/Maxwell.java @@ -9,6 +9,7 @@ import com.zendesk.maxwell.replication.BinlogConnectorReplicator; import com.zendesk.maxwell.replication.Position; import com.zendesk.maxwell.replication.Replicator; +import com.zendesk.maxwell.replication.VStreamReplicator; import com.zendesk.maxwell.row.HeartbeatRowMap; import com.zendesk.maxwell.schema.*; import com.zendesk.maxwell.schema.columndef.ColumnDefCastException; @@ -239,58 +240,72 @@ public void start() throws Exception { } private void startInner() throws Exception { - try ( Connection connection = this.context.getReplicationConnection(); - Connection rawConnection = this.context.getRawMaxwellConnection() ) { - MaxwellMysqlStatus.ensureReplicationMysqlState(connection); + try (Connection rawConnection = this.context.getRawMaxwellConnection()) { MaxwellMysqlStatus.ensureMaxwellMysqlState(rawConnection); - if (config.gtidMode) { - MaxwellMysqlStatus.ensureGtidMysqlState(connection); - } SchemaStoreSchema.ensureMaxwellSchema(rawConnection, this.config.databaseName); - try ( Connection schemaConnection = this.context.getMaxwellConnection() ) { + try (Connection schemaConnection = this.context.getMaxwellConnection()) { SchemaStoreSchema.upgradeSchemaStoreSchema(schemaConnection); } } AbstractProducer producer = this.context.getProducer(); - Position initPosition = getInitialPosition(); - logBanner(producer, initPosition); - this.context.setPosition(initPosition); - - MysqlSchemaStore mysqlSchemaStore = new MysqlSchemaStore(this.context, initPosition); - BootstrapController bootstrapController = this.context.getBootstrapController(mysqlSchemaStore.getSchemaID()); - - this.context.startSchemaCompactor(); - - if (config.recaptureSchema) { - mysqlSchemaStore.captureAndSaveSchema(); - } - - mysqlSchemaStore.getSchema(); // trigger schema to load / capture before we start the replicator. - - this.replicator = new BinlogConnectorReplicator( - mysqlSchemaStore, - producer, - bootstrapController, - config.replicationMysql, - config.replicaServerID, - config.databaseName, - context.getMetrics(), - initPosition, - false, - config.clientID, - context.getHeartbeatNotifier(), - config.scripting, - context.getFilter(), - context.getConfig().getIgnoreMissingSchema(), - config.outputConfig, - config.bufferMemoryUsage, - config.replicationReconnectionRetries, - config.binlogEventQueueSize - ); + if (config.vitessEnabled) { + this.replicator = new VStreamReplicator( + config.vitessConfig, + producer, + context.getMetrics(), + context.getFilter(), + config.bufferMemoryUsage, + config.binlogEventQueueSize + ); + } else { + try(Connection connection = context.getReplicationConnection()) { + MaxwellMysqlStatus.ensureReplicationMysqlState(connection); + + if (config.gtidMode) { + MaxwellMysqlStatus.ensureGtidMysqlState(connection); + } + } + + Position initPosition = getInitialPosition(); + logBanner(producer, initPosition); + this.context.setPosition(initPosition); + + MysqlSchemaStore mysqlSchemaStore = new MysqlSchemaStore(this.context, initPosition); + BootstrapController bootstrapController = this.context.getBootstrapController(mysqlSchemaStore.getSchemaID()); + + this.context.startSchemaCompactor(); + + if (config.recaptureSchema) { + mysqlSchemaStore.captureAndSaveSchema(); + } + + mysqlSchemaStore.getSchema(); // trigger schema to load / capture before we start the replicator. + + this.replicator = new BinlogConnectorReplicator( + mysqlSchemaStore, + producer, + bootstrapController, + config.replicationMysql, + config.replicaServerID, + config.databaseName, + context.getMetrics(), + initPosition, + false, + config.clientID, + context.getHeartbeatNotifier(), + config.scripting, + context.getFilter(), + context.getConfig().getIgnoreMissingSchema(), + config.outputConfig, + config.bufferMemoryUsage, + config.replicationReconnectionRetries, + config.binlogEventQueueSize + ); + } context.setReplicator(replicator); this.context.start(); diff --git a/src/main/java/com/zendesk/maxwell/MaxwellConfig.java b/src/main/java/com/zendesk/maxwell/MaxwellConfig.java index ede41fb42..ce554a1a6 100644 --- a/src/main/java/com/zendesk/maxwell/MaxwellConfig.java +++ b/src/main/java/com/zendesk/maxwell/MaxwellConfig.java @@ -81,6 +81,16 @@ public class MaxwellConfig extends AbstractConfig { */ public Boolean gtidMode; + /** + * If Maxwell is running against a vitess cluster. + */ + public Boolean vitessEnabled; + + /** + * Vitess (vtgate) connection config + */ + public MaxwellVitessConfig vitessConfig; + /** * Name of database in which to store maxwell data (default `maxwell`) */ @@ -641,6 +651,10 @@ public MaxwellConfig() { // argv is only null in tests this.schemaMysql = new MaxwellMysqlConfig(); this.masterRecovery = false; this.gtidMode = false; + + this.vitessEnabled = false; + this.vitessConfig = new MaxwellVitessConfig(); + this.bufferedProducerSize = 200; this.outputConfig = new MaxwellOutputConfig(); setup(null, null); // setup defaults @@ -1043,10 +1057,12 @@ private void setup(OptionSet options, Properties properties) { this.replicationMysql = parseMysqlConfig("replication_", options, properties); this.schemaMysql = parseMysqlConfig("schema_", options, properties); this.gtidMode = fetchBooleanOption("gtid_mode", options, properties, System.getenv(GTID_MODE_ENV) != null); - this.databaseName = fetchStringOption("schema_database", options, properties, "maxwell"); this.maxwellMysql.database = this.databaseName; + this.vitessEnabled = fetchBooleanOption("vitess", options, properties, false); + this.vitessConfig = parseVitessConfig(options, properties); + this.producerFactory = fetchProducerFactory(options, properties); this.producerType = fetchStringOption("producer", options, properties, "stdout"); this.producerAckTimeout = fetchLongOption("producer_ack_timeout", options, properties, 0L); diff --git a/src/main/java/com/zendesk/maxwell/MaxwellContext.java b/src/main/java/com/zendesk/maxwell/MaxwellContext.java index 1da30273f..eeeec997c 100644 --- a/src/main/java/com/zendesk/maxwell/MaxwellContext.java +++ b/src/main/java/com/zendesk/maxwell/MaxwellContext.java @@ -14,6 +14,7 @@ import com.zendesk.maxwell.schema.MysqlSchemaCompactor; import com.zendesk.maxwell.schema.PositionStoreThread; import com.zendesk.maxwell.schema.ReadOnlyMysqlPositionStore; +import com.zendesk.maxwell.schema.VitessPositionStore; import com.zendesk.maxwell.util.C3P0ConnectionPool; import com.zendesk.maxwell.util.RunLoopProcess; import com.zendesk.maxwell.util.StoppableTask; @@ -129,7 +130,9 @@ public MaxwellContext(MaxwellConfig config) throws SQLException, URISyntaxExcept if ( this.config.replayMode ) { this.positionStore = new ReadOnlyMysqlPositionStore(this.getMaxwellConnectionPool(), this.getServerID(), this.config.clientID, config.gtidMode); - } else { + } else if (this.config.vitessEnabled) { + this.positionStore = new VitessPositionStore(this.getMaxwellConnectionPool(), this.getServerID(), this.config.clientID, config.gtidMode); + } else { this.positionStore = new MysqlPositionStore(this.getMaxwellConnectionPool(), this.getServerID(), this.config.clientID, config.gtidMode); } diff --git a/src/main/java/com/zendesk/maxwell/MaxwellVitessConfig.java b/src/main/java/com/zendesk/maxwell/MaxwellVitessConfig.java new file mode 100644 index 000000000..3f3939d70 --- /dev/null +++ b/src/main/java/com/zendesk/maxwell/MaxwellVitessConfig.java @@ -0,0 +1,23 @@ +package com.zendesk.maxwell; + +public class MaxwellVitessConfig { + public String vtgateHost; + public int vtgatePort; + + public String user; + public String password; + + public String keyspace; + public String shard; + + public MaxwellVitessConfig() { + this.vtgateHost = "localhost"; + this.vtgatePort = 15991; + + this.user = null; + this.password = null; + + this.keyspace = null; + this.shard = ""; + } +} diff --git a/src/main/java/com/zendesk/maxwell/monitoring/MaxwellMetrics.java b/src/main/java/com/zendesk/maxwell/monitoring/MaxwellMetrics.java index 65c04c90d..c7c97177b 100644 --- a/src/main/java/com/zendesk/maxwell/monitoring/MaxwellMetrics.java +++ b/src/main/java/com/zendesk/maxwell/monitoring/MaxwellMetrics.java @@ -12,7 +12,7 @@ import com.zendesk.maxwell.MaxwellContext; import io.prometheus.client.CollectorRegistry; import io.prometheus.client.dropwizard.DropwizardExports; -import io.opencensus.exporter.stats.stackdriver.StackdriverStatsExporter; +// import io.opencensus.exporter.stats.stackdriver.StackdriverStatsExporter; import org.apache.commons.lang.StringUtils; import org.slf4j.Logger; @@ -126,17 +126,18 @@ private void setup(MaxwellConfig config) { } if (config.metricsReportingType.contains(reportingTypeStackdriver)) { - io.opencensus.metrics.Metrics.getExportComponent().getMetricProducerManager().add( - new io.opencensus.contrib.dropwizard.DropWizardMetrics( - Collections.singletonList(this.registry))); - - try { - StackdriverStatsExporter.createAndRegister(); - } catch (java.io.IOException e) { - LOGGER.error("Maxwell encountered an error in creating the stackdriver exporter.", e); - } - - LOGGER.info("Stackdriver metrics reporter enabled"); + throw new RuntimeException("Stackdriver metrics reporting is not supported for now"); + // io.opencensus.metrics.Metrics.getExportComponent().getMetricProducerManager().add( + // new io.opencensus.contrib.dropwizard.DropWizardMetrics( + // Collections.singletonList(this.registry))); + + // try { + // StackdriverStatsExporter.createAndRegister(); + // } catch (java.io.IOException e) { + // LOGGER.error("Maxwell encountered an error in creating the stackdriver exporter.", e); + // } + + // LOGGER.info("Stackdriver metrics reporter enabled"); } if (config.metricsReportingType.contains(reportingTypeHttp)) { diff --git a/src/main/java/com/zendesk/maxwell/replication/VStreamObserver.java b/src/main/java/com/zendesk/maxwell/replication/VStreamObserver.java new file mode 100644 index 000000000..aabb46d97 --- /dev/null +++ b/src/main/java/com/zendesk/maxwell/replication/VStreamObserver.java @@ -0,0 +1,69 @@ +package com.zendesk.maxwell.replication; + +import java.util.List; +import java.util.concurrent.LinkedBlockingDeque; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import binlogdata.Binlogdata.VEvent; +import io.grpc.Status; +import io.grpc.stub.StreamObserver; +import io.vitess.proto.Vtgate; + +/// +// Observes the VStream response stream, extracts events from VStream responses, +// then passes them to the VStreamReplicator via a queue for processing. +// +public class VStreamObserver implements StreamObserver { + private static final Logger LOGGER = LoggerFactory.getLogger(VStreamObserver.class); + private final AtomicBoolean mustStop = new AtomicBoolean(false); + private final LinkedBlockingDeque queue; + + public VStreamObserver(LinkedBlockingDeque queue) { + this.queue = queue; + } + + // Shuts down the observer + public void stop() { + mustStop.set(true); + } + + @Override + public void onNext(Vtgate.VStreamResponse response) { + LOGGER.debug("Received {} VEvents in the VStreamResponse:", response.getEventsCount()); + + List messageEvents = response.getEventsList(); + for (VEvent event : messageEvents) { + LOGGER.debug("VEvent: {}", event); + enqueueEvent(event); + } + } + + @Override + public void onError(Throwable t) { + LOGGER.error("VStream streaming onError. Status: " + Status.fromThrowable(t), t); + stop(); + } + + @Override + public void onCompleted() { + LOGGER.info("VStream streaming completed."); + stop(); + } + + // Pushes an event to the queue for VStreamReplicator to process. + private void enqueueEvent(VEvent event) { + while (mustStop.get() != true) { + try { + if (queue.offer(event, 100, TimeUnit.MILLISECONDS)) { + break; + } + } catch (InterruptedException e) { + return; + } + } + } +} diff --git a/src/main/java/com/zendesk/maxwell/replication/VStreamReplicator.java b/src/main/java/com/zendesk/maxwell/replication/VStreamReplicator.java new file mode 100644 index 000000000..3ecf8572c --- /dev/null +++ b/src/main/java/com/zendesk/maxwell/replication/VStreamReplicator.java @@ -0,0 +1,386 @@ +package com.zendesk.maxwell.replication; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.concurrent.LinkedBlockingDeque; +import java.util.concurrent.TimeUnit; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.codahale.metrics.Counter; +import com.codahale.metrics.Histogram; +import com.codahale.metrics.Meter; +import com.codahale.metrics.MetricRegistry; +import com.zendesk.maxwell.MaxwellVitessConfig; +import com.zendesk.maxwell.filtering.Filter; +import com.zendesk.maxwell.monitoring.Metrics; +import com.zendesk.maxwell.producer.AbstractProducer; +import com.zendesk.maxwell.replication.vitess.ReplicationMessageColumn; +import com.zendesk.maxwell.replication.vitess.Vgtid; +import com.zendesk.maxwell.replication.vitess.VitessSchema; +import com.zendesk.maxwell.replication.vitess.VitessTable; +import com.zendesk.maxwell.row.RowMap; +import com.zendesk.maxwell.row.RowMapBuffer; +import com.zendesk.maxwell.schema.Schema; +import com.zendesk.maxwell.schema.SchemaStoreException; +import com.zendesk.maxwell.util.RunLoopProcess; + +import binlogdata.Binlogdata.FieldEvent; +import binlogdata.Binlogdata.RowChange; +import binlogdata.Binlogdata.RowEvent; +import binlogdata.Binlogdata.VEvent; +import binlogdata.Binlogdata.VEventType; +import io.grpc.ManagedChannel; +import io.grpc.ManagedChannelBuilder; + +import io.vitess.proto.Vtgate.VStreamFlags; +import io.vitess.proto.Vtgate.VStreamRequest; +import io.vitess.proto.grpc.VitessGrpc; +import io.vitess.proto.Topodata; + + +public class VStreamReplicator extends RunLoopProcess implements Replicator { + private static final Logger LOGGER = LoggerFactory.getLogger(VStreamReplicator.class); + + private static final int GRPC_MAX_INBOUND_MESSAGE_SIZE = 4 * 1024 * 1024; + public static final int KEEPALIVE_INTERVAL_SECONDS = 60; + public static final int HEARTBEAT_INTERVAL_SECONDS = 30; + + private static final long MAX_TX_ELEMENTS = 10000; + + private static final String INSERT_TYPE = "INSERT"; + private static final String UPDATE_TYPE = "UPDATE"; + private static final String DELETE_TYPE = "DELETE"; + + private final MaxwellVitessConfig vitessConfig; + private final AbstractProducer producer; + private final Vgtid initialVgtid; + private RowMapBuffer rowBuffer; + private final float bufferMemoryUsage; + + private ManagedChannel channel; + private VStreamObserver responseObserver; + private boolean replicatorStarted = false; + private Long lastHeartbeatTime = null; + + private final LinkedBlockingDeque queue; + private final VitessSchema vitessSchema = new VitessSchema(); + private final Filter filter; + + private final Counter rowCounter; + private final Meter rowMeter; + private final Histogram transactionRowCount; + private final Histogram transactionExecutionTime; + + public VStreamReplicator( + MaxwellVitessConfig vitessConfig, + AbstractProducer producer, + Metrics metrics, + Filter filter, + Float bufferMemoryUsage, + int binlogEventQueueSize + ) { + this.vitessConfig = vitessConfig; + this.producer = producer; + this.queue = new LinkedBlockingDeque<>(binlogEventQueueSize); + this.filter = filter; + this.bufferMemoryUsage = bufferMemoryUsage; + + /* setup metrics */ + MetricRegistry mr = metrics.getRegistry(); + rowCounter = mr.counter(metrics.metricName("row", "count")); + rowMeter = mr.meter(metrics.metricName("row", "meter")); + transactionRowCount = mr.histogram(metrics.metricName("transaction", "row_count")); + transactionExecutionTime = mr.histogram(metrics.metricName("transaction", "execution_time")); + + // Providing a vgtid MySQL56/19eb2657-abc2-11ea-8ffc-0242ac11000a:1-61 here will make VStream to + // start receiving row-changes from MySQL56/19eb2657-abc2-11ea-8ffc-0242ac11000a:1-62 + // TODO: Need to load the latest vgtid from a persistent store. + initialVgtid = Vgtid.of( + List.of( + new Vgtid.ShardGtid(vitessConfig.keyspace, vitessConfig.shard, "current") + ) + ); + } + + public void startReplicator() throws Exception { + LOGGER.info( + "Starting VStreamReplicator, connecting to Vtgate at {}:{}", + vitessConfig.vtgateHost, vitessConfig.vtgatePort + ); + + this.channel = newChannel( + vitessConfig.vtgateHost, + vitessConfig.vtgatePort, + GRPC_MAX_INBOUND_MESSAGE_SIZE + ); + VitessGrpc.VitessStub stub = VitessGrpc.newStub(channel); + + VStreamFlags vStreamFlags = VStreamFlags.newBuilder() + .setStopOnReshard(true) + .setHeartbeatInterval(HEARTBEAT_INTERVAL_SECONDS) + .build(); + + VStreamRequest vstreamRequest = VStreamRequest.newBuilder() + .setVgtid(initialVgtid.getRawVgtid()) + .setTabletType(Topodata.TabletType.MASTER) + .setFlags(vStreamFlags) + .build(); + + this.responseObserver = new VStreamObserver(queue); + + stub.vStream(vstreamRequest, responseObserver); + + LOGGER.info("Started VStream"); + + this.replicatorStarted = true; + } + + @Override + protected void beforeStop() throws Exception { + responseObserver.stop(); + + channel.shutdown(); + channel.awaitTermination(500, TimeUnit.MILLISECONDS); + channel.shutdownNow(); + } + + /** + * get a single row from the replicator and pass it to the producer or bootstrapper. + * + * This is the top-level function in the run-loop. + */ + @Override + public void work() throws Exception { + RowMap row = null; + try { + row = getRow(); + } catch (InterruptedException e) { + LOGGER.debug("Interrupted while waiting for row"); + } + + if (row == null) { + return; + } + + rowCounter.inc(); + rowMeter.mark(); + + // if ( scripting != null && !isMaxwellRow(row)) + // scripting.invoke(row); + + processRow(row); + } + + public RowMap getRow() throws Exception { + if (!replicatorStarted) { + LOGGER.warn("replicator was not started, calling startReplicator()..."); + startReplicator(); + } + + while (true) { + if (rowBuffer != null && !rowBuffer.isEmpty()) { + return rowBuffer.removeFirst(); + } + + final VEvent event = pollEvent(); + if (event == null) { + return null; + } + + if (event.getType() == VEventType.BEGIN) { + rowBuffer = getTransactionRows(event); + } else { + processServiceEvent(event); + } + } + } + + private void processRow(RowMap row) throws Exception { + producer.push(row); + } + + private VEvent pollEvent() throws InterruptedException { + return queue.poll(100, TimeUnit.MILLISECONDS); + } + + private void processFieldEvent(VEvent event) { + FieldEvent fieldEvent = event.getFieldEvent(); + LOGGER.info("Received field event: " + fieldEvent); + vitessSchema.processFieldEvent(fieldEvent); + } + + + private void processVGtidEvent(VEvent event) { + LOGGER.info("Received GTID event: " + event); + } + + /** + * Get a batch of rows for the current transaction. + * + * We assume the replicator has just processed a "BEGIN" event, and now + * we're inside a transaction. We'll process all rows inside that transaction + * and turn them into RowMap objects. + * + * We do this because we want to attach all rows within + * the transaction the same transaction id value (xid, which we generate ourselves since VStream + * does not expose underlying transaction ids to the consumer). + + * @return A RowMapBuffer of rows; either in-memory or on disk. + */ + private RowMapBuffer getTransactionRows(VEvent beginEvent) throws Exception { + RowMapBuffer buffer = new RowMapBuffer(MAX_TX_ELEMENTS, this.bufferMemoryUsage); + + // Since transactions in VStream do not have an XID value, we generate one + long xid = System.currentTimeMillis() * 1000 + Math.abs(beginEvent.hashCode()) % 1000; + LOGGER.debug("Generated transaction id: {}", xid); + buffer.setXid(xid); + + while (true) { + final VEvent event = pollEvent(); + if (event == null) { + continue; + } + + final VEventType eventType = event.getType(); + + if (eventType == VEventType.COMMIT) { + LOGGER.debug("Received COMMIT event"); + if (!buffer.isEmpty()) { + buffer.getLast().setTXCommit(); + long timeSpent = buffer.getLast().getTimestampMillis() - beginEvent.getTimestamp(); + transactionExecutionTime.update(timeSpent); + transactionRowCount.update(buffer.size()); + } + return buffer; + } + + if (eventType == VEventType.ROW) { + List eventRows = rowEventToMaps(event, xid); + for (RowMap r : eventRows) { + // if (shouldOutputRowMap(table.getDatabase(), table.getName(), r, filter)) { + buffer.add(r); + } + continue; + } + + // All other events are service events delivering the schema, GTID values, etc. + processServiceEvent(event); + } + } + + private void processServiceEvent(VEvent event) { + final VEventType eventType = event.getType(); + switch(eventType) { + case FIELD: + processFieldEvent(event); + break; + + case HEARTBEAT: + LOGGER.debug("Received heartbeat from vtgate: {}", event); + break; + + case VGTID: + processVGtidEvent(event); + break; + + case ROW: + case BEGIN: + case COMMIT: + LOGGER.error("Unexpected event outside of a transaction, skipping: {}", event); + break; + + default: + LOGGER.warn("Unexpected service event: {}", event); + break; + } + + } + + private List rowEventToMaps(VEvent event, long xid) { + Long timestampMillis = event.getCurrentTime(); + RowEvent rowEvent = event.getRowEvent(); + String qualifiedTableName = rowEvent.getTableName(); + + List rowMaps = new ArrayList<>(rowEvent.getRowChangesCount()); + for (RowChange rowChange: rowEvent.getRowChangesList()) { + String changeType = rowChangeToMaxwellType(rowChange); + final VitessTable table = resolveTable(qualifiedTableName); + if (!filter.includes(table.getSchemaName(), table.getTableName())) { + LOGGER.debug("Filtering out event for table {}.{}", table.getSchemaName(), table.getTableName()); + continue; + } + List columns = table.resolveColumns(rowChange); + + RowMap rowMap = new RowMap( + changeType, + table.getSchemaName(), + table.getTableName(), + timestampMillis, + table.getPkColumns(), + null, null, null + ); + + rowMap.setXid(xid); + + for (ReplicationMessageColumn column : columns) { + rowMap.putData(column.getName(), column.getValue()); + } + + rowMaps.add(rowMap); + } + + return rowMaps; + } + + private String rowChangeToMaxwellType(RowChange change) { + if (change.hasAfter() && !change.hasBefore()) { return INSERT_TYPE; } + if (change.hasAfter() && change.hasBefore()) { return UPDATE_TYPE; } + if (!change.hasAfter() && change.hasBefore()) { return DELETE_TYPE; } + + throw new RuntimeException("Invalid row change: " + change); + } + + private VitessTable resolveTable(String qualifiedTableName) { + VitessTable table = vitessSchema.getTable(qualifiedTableName); + if (table == null) { + throw new RuntimeException("Table not found in the schema: " + qualifiedTableName); + } + return table; + } + + @Override + public Long getLastHeartbeatRead() { + // TODO Auto-generated method stub + return null; + } + + @Override + public Schema getSchema() throws SchemaStoreException { + // TODO Auto-generated method stub + return null; + } + + @Override + public Long getSchemaId() throws SchemaStoreException { + // TODO Auto-generated method stub + return null; + } + + @Override + public void stopAtHeartbeat(long heartbeat) { + // TODO Auto-generated method stub + } + + private static ManagedChannel newChannel(String vtgateHost, int vtgatePort, int maxInboundMessageSize) { + return ManagedChannelBuilder + .forAddress(vtgateHost, vtgatePort) + .usePlaintext() + .maxInboundMessageSize(maxInboundMessageSize) + .keepAliveTime(KEEPALIVE_INTERVAL_SECONDS, TimeUnit.SECONDS) + .build(); + } +} diff --git a/src/main/java/com/zendesk/maxwell/replication/vitess/ColumnMetaData.java b/src/main/java/com/zendesk/maxwell/replication/vitess/ColumnMetaData.java new file mode 100644 index 000000000..b9f293166 --- /dev/null +++ b/src/main/java/com/zendesk/maxwell/replication/vitess/ColumnMetaData.java @@ -0,0 +1,41 @@ +/* + * Copyright Debezium Authors. + * + * Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0 + */ +package com.zendesk.maxwell.replication.vitess; + +/** + * It maps the VStream FIELD to a relational column. A list of ColumnMetaData + * can be used to create + * a {@link VitessTable}. + */ +public class ColumnMetaData { + private final String columnName; + private final VitessType vitessType; + private final boolean optional; + private final KeyMetaData keyMetaData; + + public ColumnMetaData(String columnName, VitessType vitessType, boolean optional, KeyMetaData keyMetaData) { + this.columnName = columnName; + this.vitessType = vitessType; + this.keyMetaData = keyMetaData; + this.optional = optional; + } + + public String getColumnName() { + return columnName; + } + + public VitessType getVitessType() { + return vitessType; + } + + public boolean isOptional() { + return optional; + } + + public KeyMetaData getKeyMetaData() { + return keyMetaData; + } +} diff --git a/src/main/java/com/zendesk/maxwell/replication/vitess/KeyMetaData.java b/src/main/java/com/zendesk/maxwell/replication/vitess/KeyMetaData.java new file mode 100644 index 000000000..f51870c21 --- /dev/null +++ b/src/main/java/com/zendesk/maxwell/replication/vitess/KeyMetaData.java @@ -0,0 +1,26 @@ +/* + * Copyright Debezium Authors. + * + * Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0 + */ +package com.zendesk.maxwell.replication.vitess; + +/** + * If a column is part of the primary key (including multi-column primary key), + * or a column is a single-column unique key + */ +public enum KeyMetaData { + /** + * The column is part of the primary key (including multi-column primary key) + */ + IS_KEY, + /** + * The column is single-column unique key + */ + IS_UNIQUE_KEY, + /** + * The column is not part of any key, or the column is part of multi-column + * unique key + */ + NONE +} diff --git a/src/main/java/com/zendesk/maxwell/replication/vitess/ReplicationMessageColumn.java b/src/main/java/com/zendesk/maxwell/replication/vitess/ReplicationMessageColumn.java new file mode 100644 index 000000000..9211b94bb --- /dev/null +++ b/src/main/java/com/zendesk/maxwell/replication/vitess/ReplicationMessageColumn.java @@ -0,0 +1,69 @@ +/* + * Copyright Debezium Authors. + * + * Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0 + */ +package com.zendesk.maxwell.replication.vitess; + +import java.nio.charset.StandardCharsets; +import java.sql.Types; + +/** Logical representation of both column type and value. */ +public class ReplicationMessageColumn { + private final String columnName; + private final VitessType type; + private final byte[] rawValue; + + public ReplicationMessageColumn(String columnName, VitessType type, byte[] rawValue) { + this.columnName = columnName; + this.type = type; + this.rawValue = rawValue; + } + + public String getName() { + return columnName; + } + + public VitessType getType() { + return type; + } + + public Object getValue() { + final VitessColumnValue value = new VitessColumnValue(rawValue); + + if (value.isNull()) { + return null; + } + + switch (type.getJdbcId()) { + case Types.SMALLINT: + return value.asShort(); + case Types.INTEGER: + return value.asInteger(); + case Types.BIGINT: + return value.asLong(); + case Types.BLOB: + case Types.BINARY: + return value.asBytes(); + case Types.VARCHAR: + return value.asString(); + case Types.FLOAT: + return value.asFloat(); + case Types.DOUBLE: + return value.asDouble(); + default: + break; + } + + return value.asDefault(type); + } + + public byte[] getRawValue() { + return rawValue; + } + + @Override + public String toString() { + return columnName + "=" + new String(rawValue, StandardCharsets.UTF_8); + } +} diff --git a/src/main/java/com/zendesk/maxwell/replication/vitess/Vgtid.java b/src/main/java/com/zendesk/maxwell/replication/vitess/Vgtid.java new file mode 100644 index 000000000..9c94e698e --- /dev/null +++ b/src/main/java/com/zendesk/maxwell/replication/vitess/Vgtid.java @@ -0,0 +1,157 @@ +/* + * Copyright Debezium Authors. + * + * Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0 + */ +package com.zendesk.maxwell.replication.vitess; + +import java.util.ArrayList; +import java.util.List; +import java.util.Objects; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.annotation.JsonPropertyOrder; +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.ObjectMapper; + +import binlogdata.Binlogdata; + +/** Vitess source position coordinates. */ +public class Vgtid { + public static final String CURRENT_GTID = "current"; + public static final String KEYSPACE_KEY = "keyspace"; + public static final String SHARD_KEY = "shard"; + public static final String GTID_KEY = "gtid"; + + private static final ObjectMapper MAPPER = new ObjectMapper(); + + private final Binlogdata.VGtid rawVgtid; + private final List shardGtids = new ArrayList<>(); + + private Vgtid(Binlogdata.VGtid rawVgtid) { + this.rawVgtid = rawVgtid; + for (Binlogdata.ShardGtid shardGtid : rawVgtid.getShardGtidsList()) { + shardGtids.add(new ShardGtid(shardGtid.getKeyspace(), shardGtid.getShard(), shardGtid.getGtid())); + } + } + + private Vgtid(List shardGtids) { + this.shardGtids.addAll(shardGtids); + + Binlogdata.VGtid.Builder builder = Binlogdata.VGtid.newBuilder(); + for (ShardGtid shardGtid : shardGtids) { + builder.addShardGtids( + Binlogdata.ShardGtid.newBuilder() + .setKeyspace(shardGtid.getKeyspace()) + .setShard(shardGtid.getShard()) + .setGtid(shardGtid.getGtid()) + .build()); + } + this.rawVgtid = builder.build(); + } + + public static Vgtid of(String shardGtidsInJson) { + try { + List shardGtids = MAPPER.readValue(shardGtidsInJson, new TypeReference>() { + }); + return of(shardGtids); + } catch (JsonProcessingException e) { + throw new IllegalStateException(e); + } + } + + public static Vgtid of(Binlogdata.VGtid rawVgtid) { + return new Vgtid(rawVgtid); + } + + public static Vgtid of(List shardGtids) { + return new Vgtid(shardGtids); + } + + public Binlogdata.VGtid getRawVgtid() { + return rawVgtid; + } + + public List getShardGtids() { + return shardGtids; + } + + public boolean isSingleShard() { + return rawVgtid.getShardGtidsCount() == 1; + } + + @Override + public String toString() { + try { + return MAPPER.writeValueAsString(shardGtids); + } catch (JsonProcessingException e) { + throw new IllegalStateException(e); + } + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + Vgtid vgtid = (Vgtid) o; + return Objects.equals(rawVgtid, vgtid.rawVgtid) && + Objects.equals(shardGtids, vgtid.shardGtids); + } + + @Override + public int hashCode() { + return Objects.hash(rawVgtid, shardGtids); + } + + @JsonPropertyOrder({ KEYSPACE_KEY, SHARD_KEY, GTID_KEY }) + public static class ShardGtid { + private final String keyspace; + private final String shard; + private final String gtid; + + @JsonCreator + public ShardGtid(@JsonProperty(KEYSPACE_KEY) String keyspace, @JsonProperty(SHARD_KEY) String shard, + @JsonProperty(GTID_KEY) String gtid) { + this.keyspace = keyspace; + this.shard = shard; + this.gtid = gtid; + } + + public String getKeyspace() { + return keyspace; + } + + public String getShard() { + return shard; + } + + public String getGtid() { + return gtid; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + ShardGtid shardGtid = (ShardGtid) o; + return Objects.equals(keyspace, shardGtid.keyspace) && + Objects.equals(shard, shardGtid.shard) && + Objects.equals(gtid, shardGtid.gtid); + } + + @Override + public int hashCode() { + return Objects.hash(keyspace, shard, gtid); + } + } +} diff --git a/src/main/java/com/zendesk/maxwell/replication/vitess/VitessColumn.java b/src/main/java/com/zendesk/maxwell/replication/vitess/VitessColumn.java new file mode 100644 index 000000000..7ceb9a6b4 --- /dev/null +++ b/src/main/java/com/zendesk/maxwell/replication/vitess/VitessColumn.java @@ -0,0 +1,23 @@ +package com.zendesk.maxwell.replication.vitess; + +public class VitessColumn { + private final String name; + private final VitessType type; + + public VitessColumn(String name, VitessType type) { + this.name = name; + this.type = type; + } + + public String getName() { + return name; + } + + public VitessType getType() { + return type; + } + + public String toString() { + return "Column [name=" + name + ", type=" + type + "]"; + } +} diff --git a/src/main/java/com/zendesk/maxwell/replication/vitess/VitessColumnValue.java b/src/main/java/com/zendesk/maxwell/replication/vitess/VitessColumnValue.java new file mode 100644 index 000000000..1a740fd6c --- /dev/null +++ b/src/main/java/com/zendesk/maxwell/replication/vitess/VitessColumnValue.java @@ -0,0 +1,76 @@ +/* + * Copyright Debezium Authors. + * + * Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0 + */ +package com.zendesk.maxwell.replication.vitess; + +import java.nio.charset.StandardCharsets; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * A convenient wrapper that wraps the raw bytes value and converts it to Java + * value. + */ +public class VitessColumnValue { + private static final Logger LOGGER = LoggerFactory.getLogger(VitessColumnValue.class); + + private final byte[] value; + + public VitessColumnValue(byte[] value) { + this.value = value; + } + + public byte[] getRawValue() { + return value; + } + + public boolean isNull() { + return value == null; + } + + public byte[] asBytes() { + return value; + } + + /** + * Convert raw bytes value to string using UTF-8 encoding. + * + * This is *enforced* for VARCHAR and CHAR types, and is *required* for other + * non-bytes types (numeric, + * timestamp, etc.). For bytes (BLOB, BINARY, etc.) types, the asBytes() should + * be used. + * + * @return the UTF-8 string + */ + public String asString() { + return new String(value, StandardCharsets.UTF_8); + } + + public Integer asInteger() { + return Integer.valueOf(asString()); + } + + public Short asShort() { + return Short.valueOf(asString()); + } + + public Long asLong() { + return Long.valueOf(asString()); + } + + public Float asFloat() { + return Float.valueOf(asString()); + } + + public Double asDouble() { + return Double.valueOf(asString()); + } + + public Object asDefault(VitessType vitessType) { + LOGGER.warn("processing unknown column type {} as string", vitessType); + return asString(); + } +} diff --git a/src/main/java/com/zendesk/maxwell/replication/vitess/VitessSchema.java b/src/main/java/com/zendesk/maxwell/replication/vitess/VitessSchema.java new file mode 100644 index 000000000..421f4a755 --- /dev/null +++ b/src/main/java/com/zendesk/maxwell/replication/vitess/VitessSchema.java @@ -0,0 +1,125 @@ +package com.zendesk.maxwell.replication.vitess; + +import java.sql.Types; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import binlogdata.Binlogdata.FieldEvent; +import io.vitess.proto.Query.Field; + +// An in-memory representation of a Vitess schema. +public class VitessSchema { + private static final Logger LOGGER = LoggerFactory.getLogger(VitessSchema.class); + + // See all flags: + // https://dev.mysql.com/doc/dev/mysql-server/8.0.12/group__group__cs__column__definition__flags.html + private static final int NOT_NULL_FLAG = 1; + private static final int PRI_KEY_FLAG = 1 << 1; + private static final int UNIQUE_KEY_FLAG = 1 << 2; + + private final Map tables = new HashMap<>(); + + public void addTable(VitessTable table) { + String tableName = table.getQualifiedTableName(); + if (tables.containsKey(tableName)) { + LOGGER.info("Schema change detected for: {}", table); + } else { + LOGGER.info("Table schema received for: {}", table); + } + tables.put(tableName, table); + } + + public VitessTable getTable(String name) { + return tables.get(name); + } + + public void processFieldEvent(FieldEvent event) { + if (event == null) { + throw new RuntimeException(String.format("fieldEvent is expected from {}", event)); + } + + String qualifiedTableName = event.getTableName(); + String[] schemaTableTuple = qualifiedTableName.split("\\."); + if (schemaTableTuple.length != 2) { + throw new RuntimeException( + String.format( + "Handling FIELD VEvent. schemaTableTuple should have schema name and table name but has size {}. {} is skipped", + schemaTableTuple.length, + event)); + } + + LOGGER.debug("Handling FIELD VEvent: {}", event); + String schemaName = schemaTableTuple[0]; + String tableName = schemaTableTuple[1]; + int columnCount = event.getFieldsCount(); + + List columns = new ArrayList<>(columnCount); + for (short i = 0; i < columnCount; ++i) { + Field field = event.getFields(i); + String columnName = validateColumnName(field.getName(), schemaName, tableName); + + VitessType vitessType = VitessType.resolve(field); + if (vitessType.getJdbcId() == Types.OTHER) { + LOGGER.error("Cannot resolve JDBC type from VStream field {}", field); + } + + KeyMetaData keyMetaData = KeyMetaData.NONE; + if ((field.getFlags() & PRI_KEY_FLAG) != 0) { + keyMetaData = KeyMetaData.IS_KEY; + } else if ((field.getFlags() & UNIQUE_KEY_FLAG) != 0) { + keyMetaData = KeyMetaData.IS_UNIQUE_KEY; + } + boolean optional = (field.getFlags() & NOT_NULL_FLAG) == 0; + + columns.add(new ColumnMetaData(columnName, vitessType, optional, keyMetaData)); + } + + VitessTable table = createTable(schemaName, tableName, columns); + addTable(table); + } + + private static String validateColumnName(String columnName, String schemaName, String tableName) { + int length = columnName.length(); + if (length == 0) { + throw new IllegalArgumentException( + String.format("Empty column name from schema: %s, table: %s", schemaName, tableName)); + } + + // Vitess VStreamer schema reloading transient bug could cause column names to + // be anonymized to @1, @2, etc + // We want to fail in this case instead of sending the corrupted row events with + // @1, @2 as column names. + char first = columnName.charAt(0); + if (first == '@') { + throw new IllegalArgumentException( + String.format( + "Illegal prefix '@' for column: %s, from schema: %s, table: %s", + columnName, + schemaName, + tableName)); + } + + return columnName; + } + + private VitessTable createTable(String schemaName, String tableName, List columnsMetaData) { + List columns = new ArrayList<>(columnsMetaData.size()); + List pkColumns = new ArrayList<>(); + + for (ColumnMetaData columnMetaData : columnsMetaData) { + VitessColumn column = new VitessColumn(columnMetaData.getColumnName(), columnMetaData.getVitessType()); + columns.add(column); + + if (columnMetaData.getKeyMetaData() == KeyMetaData.IS_KEY) { + pkColumns.add(columnMetaData.getColumnName()); + } + } + + return new VitessTable(schemaName, tableName, columns, pkColumns); + } +} diff --git a/src/main/java/com/zendesk/maxwell/replication/vitess/VitessTable.java b/src/main/java/com/zendesk/maxwell/replication/vitess/VitessTable.java new file mode 100644 index 000000000..e9ed928b0 --- /dev/null +++ b/src/main/java/com/zendesk/maxwell/replication/vitess/VitessTable.java @@ -0,0 +1,89 @@ +package com.zendesk.maxwell.replication.vitess; + +import java.util.ArrayList; +import java.util.List; + +import com.google.protobuf.ByteString; + +import binlogdata.Binlogdata.RowChange; +import io.vitess.proto.Query.Row; + +public class VitessTable { + private final String schemaName; + private final String tableName; + private final List columns; + private final List pkColumns; + + public VitessTable(String schemaName, String tableName, List columns, List pkColumns) { + this.schemaName = schemaName; + this.tableName = tableName; + this.columns = columns; + this.pkColumns = pkColumns; + } + + public String getSchemaName() { + return schemaName; + } + + public String getTableName() { + return tableName; + } + + public String getQualifiedTableName() { + return schemaName + "." + tableName; + } + + public List getColumns() { + return columns; + } + + public List getPkColumns() { + return pkColumns; + } + + public String toString() { + return "Table [schemaName=" + schemaName + + ", tableName=" + tableName + + ", columns=" + columns + + ", pkColumns=" + pkColumns + + "]"; + } + + /** + * Resolve the vEvent data to a list of replication message columns (with + * values). + */ + public List resolveColumns(RowChange rowChange) { + Row row = rowChange.hasAfter() ? rowChange.getAfter() : rowChange.getBefore(); + int changedColumnsCnt = row.getLengthsCount(); + if (columns.size() != changedColumnsCnt) { + throw new IllegalStateException( + String.format( + "The number of columns in the ROW event {} is different from the in-memory table schema {}.", + row, + this)); + } + + ByteString rawValues = row.getValues(); + int rawValueIndex = 0; + List eventColumns = new ArrayList<>(changedColumnsCnt); + for (short i = 0; i < changedColumnsCnt; i++) { + final VitessColumn columnDefinition = columns.get(i); + final String columnName = columnDefinition.getName(); + final VitessType vitessType = columnDefinition.getType(); + + final int rawValueLength = (int) row.getLengths(i); + final byte[] rawValue = rawValueLength == -1 + ? null + : rawValues.substring(rawValueIndex, rawValueIndex + rawValueLength).toByteArray(); + if (rawValueLength != -1) { + // no update to rawValueIndex when no value in the rawValue + rawValueIndex += rawValueLength; + } + + final ReplicationMessageColumn column = new ReplicationMessageColumn(columnName, vitessType, rawValue); + eventColumns.add(column); + } + return eventColumns; + } +} diff --git a/src/main/java/com/zendesk/maxwell/replication/vitess/VitessType.java b/src/main/java/com/zendesk/maxwell/replication/vitess/VitessType.java new file mode 100644 index 000000000..68cd4e106 --- /dev/null +++ b/src/main/java/com/zendesk/maxwell/replication/vitess/VitessType.java @@ -0,0 +1,163 @@ +/* + * Copyright Debezium Authors. + * + * Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0 + */ +package com.zendesk.maxwell.replication.vitess; + +import java.sql.Types; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Objects; + +import io.vitess.proto.Query.Field; + +/** The Vitess table column type */ +public class VitessType { + + // name of the column type + private final String name; + // enum of column jdbc type + private final int jdbcId; + // permitted enum values + private final List enumValues; + + public VitessType(String name, int jdbcId) { + this(name, jdbcId, Collections.emptyList()); + } + + public VitessType(String name, int jdbcId, List enumValues) { + this.name = name; + this.jdbcId = jdbcId; + this.enumValues = Collections.unmodifiableList(enumValues); + } + + public String getName() { + return name; + } + + public int getJdbcId() { + return jdbcId; + } + + public List getEnumValues() { + return enumValues; + } + + public boolean isEnum() { + return !enumValues.isEmpty(); + } + + @Override + public String toString() { + return "VitessType{" + + "name='" + name + '\'' + + ", jdbcId=" + jdbcId + + ", enumValues=" + enumValues + + '}'; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + VitessType that = (VitessType) o; + return jdbcId == that.jdbcId && name.equals(that.name) && Objects.equals(enumValues, that.enumValues); + } + + @Override + public int hashCode() { + return Objects.hash(name, jdbcId, enumValues); + } + + // Resolve JDBC type from vstream FIELD event + public static VitessType resolve(Field field) { + final String type = field.getType().name(); + switch (type) { + case "INT8": + case "UINT8": + case "INT16": + return new VitessType(type, Types.SMALLINT); + case "UINT16": + case "INT24": + case "UINT24": + case "INT32": + return new VitessType(type, Types.INTEGER); + case "ENUM": + return new VitessType(type, Types.INTEGER, resolveEnumAndSetValues(field.getColumnType())); + case "SET": + return new VitessType(type, Types.BIGINT, resolveEnumAndSetValues(field.getColumnType())); + case "UINT32": + case "INT64": + return new VitessType(type, Types.BIGINT); + case "BLOB": + return new VitessType(type, Types.BLOB); + case "VARBINARY": + case "BINARY": + return new VitessType(type, Types.BINARY); + case "UINT64": + case "VARCHAR": + case "CHAR": + case "TEXT": + case "JSON": + case "DECIMAL": + case "TIME": + case "DATE": + case "DATETIME": + case "TIMESTAMP": + case "YEAR": + return new VitessType(type, Types.VARCHAR); + case "FLOAT32": + return new VitessType(type, Types.FLOAT); + case "FLOAT64": + return new VitessType(type, Types.DOUBLE); + default: + return new VitessType(type, Types.OTHER); + } + } + + /** + * Resolve the list of permitted Enum or Set values from the Enum or Set + * Definition + * + * @param definition the Enum or Set column definition from the MySQL table. + * E.g. "enum('m','l','xl')" or "set('a','b','c')" + * @return The list of permitted Enum values or Set values + */ + private static List resolveEnumAndSetValues(String definition) { + List values = new ArrayList<>(); + if (definition == null || definition.length() == 0) { + return values; + } + + StringBuilder sb = new StringBuilder(); + boolean startCollecting = false; + char[] chars = definition.toCharArray(); + for (int i = 0; i < chars.length; i++) { + if (chars[i] == '\'') { + if (chars[i + 1] != '\'') { + if (startCollecting) { + // end of the Enum/Set value, add the Enum/Set value to the result list + values.add(sb.toString()); + sb.setLength(0); + } + startCollecting = !startCollecting; + } else { + sb.append("'"); + // In MySQL, the single quote in the Enum/Set definition "a'b" is escaped and + // becomes "a''b". + // Skip the second single-quote + i++; + } + } else if (startCollecting) { + sb.append(chars[i]); + } + } + return values; + } +} diff --git a/src/main/java/com/zendesk/maxwell/schema/VitessPositionStore.java b/src/main/java/com/zendesk/maxwell/schema/VitessPositionStore.java new file mode 100644 index 000000000..8877c1b8b --- /dev/null +++ b/src/main/java/com/zendesk/maxwell/schema/VitessPositionStore.java @@ -0,0 +1,20 @@ +package com.zendesk.maxwell.schema; + +import com.zendesk.maxwell.replication.Position; +import com.zendesk.maxwell.util.ConnectionPool; + +public class VitessPositionStore extends MysqlPositionStore { + public VitessPositionStore(ConnectionPool pool, Long serverID, String clientID, boolean gtidMode) { + super(pool, serverID, clientID, gtidMode); + } + + @Override + public void set(Position p) { + // TODO: implement this for storing vtgid values + } + + @Override + public long heartbeat() throws Exception { + return System.currentTimeMillis(); + } +} diff --git a/src/main/java/com/zendesk/maxwell/util/AbstractConfig.java b/src/main/java/com/zendesk/maxwell/util/AbstractConfig.java index 028a4b934..d8f5dcdbf 100644 --- a/src/main/java/com/zendesk/maxwell/util/AbstractConfig.java +++ b/src/main/java/com/zendesk/maxwell/util/AbstractConfig.java @@ -12,6 +12,8 @@ import joptsimple.*; import com.zendesk.maxwell.MaxwellMysqlConfig; +import com.zendesk.maxwell.MaxwellVitessConfig; + import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -224,6 +226,17 @@ protected MaxwellMysqlConfig parseMysqlConfig(String prefix, OptionSet options, return config; } + protected MaxwellVitessConfig parseVitessConfig(OptionSet options, Properties properties) { + MaxwellVitessConfig config = new MaxwellVitessConfig(); + config.vtgateHost = fetchStringOption("vitess_host", options, properties, "localhost"); + config.vtgatePort = fetchIntegerOption("vitess_port", options, properties, 15991); + config.user = fetchStringOption("vitess_user", options, properties, null); + config.password = fetchStringOption("vitess_password", options, properties, null); + config.keyspace = fetchStringOption("vitess_keyspace", options, properties, null); + config.shard = fetchStringOption("vitess_shard", options, properties, ""); + return config; + } + private SSLMode getSslModeFromString(String sslMode) { if (sslMode != null) { for (SSLMode mode : SSLMode.values()) { From b1d7b7729ce7fa53aeaa77b411d0ff60844b9ce1 Mon Sep 17 00:00:00 2001 From: Oleksiy Kovyrin Date: Mon, 21 Nov 2022 11:49:03 -0500 Subject: [PATCH 02/32] Switch to using tabs instead of spaces --- .../com/zendesk/maxwell/MaxwellConfig.java | 8 +- .../zendesk/maxwell/MaxwellVitessConfig.java | 28 +- .../maxwell/replication/VStreamObserver.java | 82 +-- .../replication/VStreamReplicator.java | 685 +++++++++--------- .../replication/vitess/ColumnMetaData.java | 44 +- .../replication/vitess/KeyMetaData.java | 26 +- .../vitess/ReplicationMessageColumn.java | 94 +-- .../maxwell/replication/vitess/Vgtid.java | 268 +++---- .../replication/vitess/VitessColumn.java | 30 +- .../replication/vitess/VitessColumnValue.java | 92 +-- .../replication/vitess/VitessSchema.java | 216 +++--- .../replication/vitess/VitessTable.java | 132 ++-- .../replication/vitess/VitessType.java | 289 ++++---- 13 files changed, 997 insertions(+), 997 deletions(-) diff --git a/src/main/java/com/zendesk/maxwell/MaxwellConfig.java b/src/main/java/com/zendesk/maxwell/MaxwellConfig.java index ce554a1a6..1750a28f6 100644 --- a/src/main/java/com/zendesk/maxwell/MaxwellConfig.java +++ b/src/main/java/com/zendesk/maxwell/MaxwellConfig.java @@ -652,8 +652,8 @@ public MaxwellConfig() { // argv is only null in tests this.masterRecovery = false; this.gtidMode = false; - this.vitessEnabled = false; - this.vitessConfig = new MaxwellVitessConfig(); + this.vitessEnabled = false; + this.vitessConfig = new MaxwellVitessConfig(); this.bufferedProducerSize = 200; this.outputConfig = new MaxwellOutputConfig(); @@ -1060,8 +1060,8 @@ private void setup(OptionSet options, Properties properties) { this.databaseName = fetchStringOption("schema_database", options, properties, "maxwell"); this.maxwellMysql.database = this.databaseName; - this.vitessEnabled = fetchBooleanOption("vitess", options, properties, false); - this.vitessConfig = parseVitessConfig(options, properties); + this.vitessEnabled = fetchBooleanOption("vitess", options, properties, false); + this.vitessConfig = parseVitessConfig(options, properties); this.producerFactory = fetchProducerFactory(options, properties); this.producerType = fetchStringOption("producer", options, properties, "stdout"); diff --git a/src/main/java/com/zendesk/maxwell/MaxwellVitessConfig.java b/src/main/java/com/zendesk/maxwell/MaxwellVitessConfig.java index 3f3939d70..655aa4899 100644 --- a/src/main/java/com/zendesk/maxwell/MaxwellVitessConfig.java +++ b/src/main/java/com/zendesk/maxwell/MaxwellVitessConfig.java @@ -1,23 +1,23 @@ package com.zendesk.maxwell; public class MaxwellVitessConfig { - public String vtgateHost; - public int vtgatePort; + public String vtgateHost; + public int vtgatePort; - public String user; - public String password; + public String user; + public String password; - public String keyspace; - public String shard; + public String keyspace; + public String shard; - public MaxwellVitessConfig() { - this.vtgateHost = "localhost"; - this.vtgatePort = 15991; + public MaxwellVitessConfig() { + this.vtgateHost = "localhost"; + this.vtgatePort = 15991; - this.user = null; - this.password = null; + this.user = null; + this.password = null; - this.keyspace = null; - this.shard = ""; - } + this.keyspace = null; + this.shard = ""; + } } diff --git a/src/main/java/com/zendesk/maxwell/replication/VStreamObserver.java b/src/main/java/com/zendesk/maxwell/replication/VStreamObserver.java index aabb46d97..97d16c06a 100644 --- a/src/main/java/com/zendesk/maxwell/replication/VStreamObserver.java +++ b/src/main/java/com/zendesk/maxwell/replication/VStreamObserver.java @@ -18,52 +18,52 @@ // then passes them to the VStreamReplicator via a queue for processing. // public class VStreamObserver implements StreamObserver { - private static final Logger LOGGER = LoggerFactory.getLogger(VStreamObserver.class); - private final AtomicBoolean mustStop = new AtomicBoolean(false); - private final LinkedBlockingDeque queue; + private static final Logger LOGGER = LoggerFactory.getLogger(VStreamObserver.class); + private final AtomicBoolean mustStop = new AtomicBoolean(false); + private final LinkedBlockingDeque queue; - public VStreamObserver(LinkedBlockingDeque queue) { - this.queue = queue; - } + public VStreamObserver(LinkedBlockingDeque queue) { + this.queue = queue; + } - // Shuts down the observer - public void stop() { - mustStop.set(true); - } + // Shuts down the observer + public void stop() { + mustStop.set(true); + } - @Override - public void onNext(Vtgate.VStreamResponse response) { - LOGGER.debug("Received {} VEvents in the VStreamResponse:", response.getEventsCount()); + @Override + public void onNext(Vtgate.VStreamResponse response) { + LOGGER.debug("Received {} VEvents in the VStreamResponse:", response.getEventsCount()); - List messageEvents = response.getEventsList(); - for (VEvent event : messageEvents) { - LOGGER.debug("VEvent: {}", event); - enqueueEvent(event); - } - } + List messageEvents = response.getEventsList(); + for (VEvent event : messageEvents) { + LOGGER.debug("VEvent: {}", event); + enqueueEvent(event); + } + } - @Override - public void onError(Throwable t) { - LOGGER.error("VStream streaming onError. Status: " + Status.fromThrowable(t), t); - stop(); - } + @Override + public void onError(Throwable t) { + LOGGER.error("VStream streaming onError. Status: " + Status.fromThrowable(t), t); + stop(); + } - @Override - public void onCompleted() { - LOGGER.info("VStream streaming completed."); - stop(); - } + @Override + public void onCompleted() { + LOGGER.info("VStream streaming completed."); + stop(); + } - // Pushes an event to the queue for VStreamReplicator to process. - private void enqueueEvent(VEvent event) { - while (mustStop.get() != true) { - try { - if (queue.offer(event, 100, TimeUnit.MILLISECONDS)) { - break; - } - } catch (InterruptedException e) { - return; - } - } - } + // Pushes an event to the queue for VStreamReplicator to process. + private void enqueueEvent(VEvent event) { + while (mustStop.get() != true) { + try { + if (queue.offer(event, 100, TimeUnit.MILLISECONDS)) { + break; + } + } catch (InterruptedException e) { + return; + } + } + } } diff --git a/src/main/java/com/zendesk/maxwell/replication/VStreamReplicator.java b/src/main/java/com/zendesk/maxwell/replication/VStreamReplicator.java index 3ecf8572c..6fcc1b63f 100644 --- a/src/main/java/com/zendesk/maxwell/replication/VStreamReplicator.java +++ b/src/main/java/com/zendesk/maxwell/replication/VStreamReplicator.java @@ -1,8 +1,6 @@ package com.zendesk.maxwell.replication; import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; import java.util.List; import java.util.concurrent.LinkedBlockingDeque; import java.util.concurrent.TimeUnit; @@ -41,346 +39,347 @@ import io.vitess.proto.grpc.VitessGrpc; import io.vitess.proto.Topodata; - public class VStreamReplicator extends RunLoopProcess implements Replicator { - private static final Logger LOGGER = LoggerFactory.getLogger(VStreamReplicator.class); - - private static final int GRPC_MAX_INBOUND_MESSAGE_SIZE = 4 * 1024 * 1024; - public static final int KEEPALIVE_INTERVAL_SECONDS = 60; - public static final int HEARTBEAT_INTERVAL_SECONDS = 30; - - private static final long MAX_TX_ELEMENTS = 10000; - - private static final String INSERT_TYPE = "INSERT"; - private static final String UPDATE_TYPE = "UPDATE"; - private static final String DELETE_TYPE = "DELETE"; - - private final MaxwellVitessConfig vitessConfig; - private final AbstractProducer producer; - private final Vgtid initialVgtid; - private RowMapBuffer rowBuffer; - private final float bufferMemoryUsage; - - private ManagedChannel channel; - private VStreamObserver responseObserver; - private boolean replicatorStarted = false; - private Long lastHeartbeatTime = null; - - private final LinkedBlockingDeque queue; - private final VitessSchema vitessSchema = new VitessSchema(); - private final Filter filter; - - private final Counter rowCounter; - private final Meter rowMeter; - private final Histogram transactionRowCount; - private final Histogram transactionExecutionTime; - - public VStreamReplicator( - MaxwellVitessConfig vitessConfig, - AbstractProducer producer, - Metrics metrics, - Filter filter, - Float bufferMemoryUsage, - int binlogEventQueueSize - ) { - this.vitessConfig = vitessConfig; - this.producer = producer; - this.queue = new LinkedBlockingDeque<>(binlogEventQueueSize); - this.filter = filter; - this.bufferMemoryUsage = bufferMemoryUsage; - - /* setup metrics */ - MetricRegistry mr = metrics.getRegistry(); - rowCounter = mr.counter(metrics.metricName("row", "count")); - rowMeter = mr.meter(metrics.metricName("row", "meter")); - transactionRowCount = mr.histogram(metrics.metricName("transaction", "row_count")); - transactionExecutionTime = mr.histogram(metrics.metricName("transaction", "execution_time")); - - // Providing a vgtid MySQL56/19eb2657-abc2-11ea-8ffc-0242ac11000a:1-61 here will make VStream to - // start receiving row-changes from MySQL56/19eb2657-abc2-11ea-8ffc-0242ac11000a:1-62 - // TODO: Need to load the latest vgtid from a persistent store. - initialVgtid = Vgtid.of( - List.of( - new Vgtid.ShardGtid(vitessConfig.keyspace, vitessConfig.shard, "current") - ) - ); - } - - public void startReplicator() throws Exception { - LOGGER.info( - "Starting VStreamReplicator, connecting to Vtgate at {}:{}", - vitessConfig.vtgateHost, vitessConfig.vtgatePort - ); - - this.channel = newChannel( - vitessConfig.vtgateHost, - vitessConfig.vtgatePort, - GRPC_MAX_INBOUND_MESSAGE_SIZE - ); - VitessGrpc.VitessStub stub = VitessGrpc.newStub(channel); - - VStreamFlags vStreamFlags = VStreamFlags.newBuilder() - .setStopOnReshard(true) - .setHeartbeatInterval(HEARTBEAT_INTERVAL_SECONDS) - .build(); - - VStreamRequest vstreamRequest = VStreamRequest.newBuilder() - .setVgtid(initialVgtid.getRawVgtid()) - .setTabletType(Topodata.TabletType.MASTER) - .setFlags(vStreamFlags) - .build(); - - this.responseObserver = new VStreamObserver(queue); - - stub.vStream(vstreamRequest, responseObserver); - - LOGGER.info("Started VStream"); - - this.replicatorStarted = true; - } - - @Override - protected void beforeStop() throws Exception { - responseObserver.stop(); - - channel.shutdown(); - channel.awaitTermination(500, TimeUnit.MILLISECONDS); - channel.shutdownNow(); - } - - /** - * get a single row from the replicator and pass it to the producer or bootstrapper. - * - * This is the top-level function in the run-loop. - */ - @Override - public void work() throws Exception { - RowMap row = null; - try { - row = getRow(); - } catch (InterruptedException e) { - LOGGER.debug("Interrupted while waiting for row"); - } - - if (row == null) { - return; - } - - rowCounter.inc(); - rowMeter.mark(); - - // if ( scripting != null && !isMaxwellRow(row)) - // scripting.invoke(row); - - processRow(row); - } - - public RowMap getRow() throws Exception { - if (!replicatorStarted) { - LOGGER.warn("replicator was not started, calling startReplicator()..."); - startReplicator(); - } - - while (true) { - if (rowBuffer != null && !rowBuffer.isEmpty()) { - return rowBuffer.removeFirst(); - } - - final VEvent event = pollEvent(); - if (event == null) { - return null; - } - - if (event.getType() == VEventType.BEGIN) { - rowBuffer = getTransactionRows(event); - } else { - processServiceEvent(event); - } - } - } - - private void processRow(RowMap row) throws Exception { - producer.push(row); - } - - private VEvent pollEvent() throws InterruptedException { - return queue.poll(100, TimeUnit.MILLISECONDS); - } - - private void processFieldEvent(VEvent event) { - FieldEvent fieldEvent = event.getFieldEvent(); - LOGGER.info("Received field event: " + fieldEvent); - vitessSchema.processFieldEvent(fieldEvent); - } - - - private void processVGtidEvent(VEvent event) { - LOGGER.info("Received GTID event: " + event); - } - - /** - * Get a batch of rows for the current transaction. - * - * We assume the replicator has just processed a "BEGIN" event, and now - * we're inside a transaction. We'll process all rows inside that transaction - * and turn them into RowMap objects. - * - * We do this because we want to attach all rows within - * the transaction the same transaction id value (xid, which we generate ourselves since VStream - * does not expose underlying transaction ids to the consumer). - - * @return A RowMapBuffer of rows; either in-memory or on disk. - */ - private RowMapBuffer getTransactionRows(VEvent beginEvent) throws Exception { - RowMapBuffer buffer = new RowMapBuffer(MAX_TX_ELEMENTS, this.bufferMemoryUsage); - - // Since transactions in VStream do not have an XID value, we generate one - long xid = System.currentTimeMillis() * 1000 + Math.abs(beginEvent.hashCode()) % 1000; - LOGGER.debug("Generated transaction id: {}", xid); - buffer.setXid(xid); - - while (true) { - final VEvent event = pollEvent(); - if (event == null) { - continue; - } - - final VEventType eventType = event.getType(); - - if (eventType == VEventType.COMMIT) { - LOGGER.debug("Received COMMIT event"); - if (!buffer.isEmpty()) { - buffer.getLast().setTXCommit(); - long timeSpent = buffer.getLast().getTimestampMillis() - beginEvent.getTimestamp(); - transactionExecutionTime.update(timeSpent); - transactionRowCount.update(buffer.size()); - } - return buffer; - } - - if (eventType == VEventType.ROW) { - List eventRows = rowEventToMaps(event, xid); - for (RowMap r : eventRows) { - // if (shouldOutputRowMap(table.getDatabase(), table.getName(), r, filter)) { - buffer.add(r); - } - continue; - } - - // All other events are service events delivering the schema, GTID values, etc. - processServiceEvent(event); - } - } - - private void processServiceEvent(VEvent event) { - final VEventType eventType = event.getType(); - switch(eventType) { - case FIELD: - processFieldEvent(event); - break; - - case HEARTBEAT: - LOGGER.debug("Received heartbeat from vtgate: {}", event); - break; - - case VGTID: - processVGtidEvent(event); - break; - - case ROW: - case BEGIN: - case COMMIT: - LOGGER.error("Unexpected event outside of a transaction, skipping: {}", event); - break; - - default: - LOGGER.warn("Unexpected service event: {}", event); - break; - } - - } - - private List rowEventToMaps(VEvent event, long xid) { - Long timestampMillis = event.getCurrentTime(); - RowEvent rowEvent = event.getRowEvent(); - String qualifiedTableName = rowEvent.getTableName(); - - List rowMaps = new ArrayList<>(rowEvent.getRowChangesCount()); - for (RowChange rowChange: rowEvent.getRowChangesList()) { - String changeType = rowChangeToMaxwellType(rowChange); - final VitessTable table = resolveTable(qualifiedTableName); - if (!filter.includes(table.getSchemaName(), table.getTableName())) { - LOGGER.debug("Filtering out event for table {}.{}", table.getSchemaName(), table.getTableName()); - continue; - } - List columns = table.resolveColumns(rowChange); - - RowMap rowMap = new RowMap( - changeType, - table.getSchemaName(), - table.getTableName(), - timestampMillis, - table.getPkColumns(), - null, null, null - ); - - rowMap.setXid(xid); - - for (ReplicationMessageColumn column : columns) { - rowMap.putData(column.getName(), column.getValue()); - } - - rowMaps.add(rowMap); - } - - return rowMaps; - } - - private String rowChangeToMaxwellType(RowChange change) { - if (change.hasAfter() && !change.hasBefore()) { return INSERT_TYPE; } - if (change.hasAfter() && change.hasBefore()) { return UPDATE_TYPE; } - if (!change.hasAfter() && change.hasBefore()) { return DELETE_TYPE; } - - throw new RuntimeException("Invalid row change: " + change); - } - - private VitessTable resolveTable(String qualifiedTableName) { - VitessTable table = vitessSchema.getTable(qualifiedTableName); - if (table == null) { - throw new RuntimeException("Table not found in the schema: " + qualifiedTableName); - } - return table; - } - - @Override - public Long getLastHeartbeatRead() { - // TODO Auto-generated method stub - return null; - } - - @Override - public Schema getSchema() throws SchemaStoreException { - // TODO Auto-generated method stub - return null; - } - - @Override - public Long getSchemaId() throws SchemaStoreException { - // TODO Auto-generated method stub - return null; - } - - @Override - public void stopAtHeartbeat(long heartbeat) { - // TODO Auto-generated method stub - } - - private static ManagedChannel newChannel(String vtgateHost, int vtgatePort, int maxInboundMessageSize) { - return ManagedChannelBuilder - .forAddress(vtgateHost, vtgatePort) - .usePlaintext() - .maxInboundMessageSize(maxInboundMessageSize) - .keepAliveTime(KEEPALIVE_INTERVAL_SECONDS, TimeUnit.SECONDS) - .build(); - } + private static final Logger LOGGER = LoggerFactory.getLogger(VStreamReplicator.class); + + private static final int GRPC_MAX_INBOUND_MESSAGE_SIZE = 4 * 1024 * 1024; + public static final int KEEPALIVE_INTERVAL_SECONDS = 60; + public static final int HEARTBEAT_INTERVAL_SECONDS = 30; + + private static final long MAX_TX_ELEMENTS = 10000; + + private static final String INSERT_TYPE = "INSERT"; + private static final String UPDATE_TYPE = "UPDATE"; + private static final String DELETE_TYPE = "DELETE"; + + private final MaxwellVitessConfig vitessConfig; + private final AbstractProducer producer; + private final Vgtid initialVgtid; + private RowMapBuffer rowBuffer; + private final float bufferMemoryUsage; + + private ManagedChannel channel; + private VStreamObserver responseObserver; + private boolean replicatorStarted = false; + + private final LinkedBlockingDeque queue; + private final VitessSchema vitessSchema = new VitessSchema(); + private final Filter filter; + + private final Counter rowCounter; + private final Meter rowMeter; + private final Histogram transactionRowCount; + private final Histogram transactionExecutionTime; + + public VStreamReplicator( + MaxwellVitessConfig vitessConfig, + AbstractProducer producer, + Metrics metrics, + Filter filter, + Float bufferMemoryUsage, + int binlogEventQueueSize) { + this.vitessConfig = vitessConfig; + this.producer = producer; + this.queue = new LinkedBlockingDeque<>(binlogEventQueueSize); + this.filter = filter; + this.bufferMemoryUsage = bufferMemoryUsage; + + /* setup metrics */ + MetricRegistry mr = metrics.getRegistry(); + rowCounter = mr.counter(metrics.metricName("row", "count")); + rowMeter = mr.meter(metrics.metricName("row", "meter")); + transactionRowCount = mr.histogram(metrics.metricName("transaction", "row_count")); + transactionExecutionTime = mr.histogram(metrics.metricName("transaction", "execution_time")); + + // Providing a vgtid MySQL56/19eb2657-abc2-11ea-8ffc-0242ac11000a:1-61 here will + // make VStream to + // start receiving row-changes from + // MySQL56/19eb2657-abc2-11ea-8ffc-0242ac11000a:1-62 + // TODO: Need to load the latest vgtid from a persistent store. + initialVgtid = Vgtid.of( + List.of( + new Vgtid.ShardGtid(vitessConfig.keyspace, vitessConfig.shard, "current"))); + } + + public void startReplicator() throws Exception { + LOGGER.info( + "Starting VStreamReplicator, connecting to Vtgate at {}:{}", + vitessConfig.vtgateHost, vitessConfig.vtgatePort); + + this.channel = newChannel( + vitessConfig.vtgateHost, + vitessConfig.vtgatePort, + GRPC_MAX_INBOUND_MESSAGE_SIZE); + VitessGrpc.VitessStub stub = VitessGrpc.newStub(channel); + + VStreamFlags vStreamFlags = VStreamFlags.newBuilder() + .setStopOnReshard(true) + .setHeartbeatInterval(HEARTBEAT_INTERVAL_SECONDS) + .build(); + + VStreamRequest vstreamRequest = VStreamRequest.newBuilder() + .setVgtid(initialVgtid.getRawVgtid()) + .setTabletType(Topodata.TabletType.MASTER) + .setFlags(vStreamFlags) + .build(); + + this.responseObserver = new VStreamObserver(queue); + + stub.vStream(vstreamRequest, responseObserver); + + LOGGER.info("Started VStream"); + + this.replicatorStarted = true; + } + + @Override + protected void beforeStop() throws Exception { + responseObserver.stop(); + + channel.shutdown(); + channel.awaitTermination(500, TimeUnit.MILLISECONDS); + channel.shutdownNow(); + } + + /** + * get a single row from the replicator and pass it to the producer or + * bootstrapper. + * + * This is the top-level function in the run-loop. + */ + @Override + public void work() throws Exception { + RowMap row = null; + try { + row = getRow(); + } catch (InterruptedException e) { + LOGGER.debug("Interrupted while waiting for row"); + } + + if (row == null) { + return; + } + + rowCounter.inc(); + rowMeter.mark(); + + // if ( scripting != null && !isMaxwellRow(row)) + // scripting.invoke(row); + + processRow(row); + } + + public RowMap getRow() throws Exception { + if (!replicatorStarted) { + LOGGER.warn("replicator was not started, calling startReplicator()..."); + startReplicator(); + } + + while (true) { + if (rowBuffer != null && !rowBuffer.isEmpty()) { + return rowBuffer.removeFirst(); + } + + final VEvent event = pollEvent(); + if (event == null) { + return null; + } + + if (event.getType() == VEventType.BEGIN) { + rowBuffer = getTransactionRows(event); + } else { + processServiceEvent(event); + } + } + } + + private void processRow(RowMap row) throws Exception { + producer.push(row); + } + + private VEvent pollEvent() throws InterruptedException { + return queue.poll(100, TimeUnit.MILLISECONDS); + } + + private void processFieldEvent(VEvent event) { + FieldEvent fieldEvent = event.getFieldEvent(); + LOGGER.info("Received field event: " + fieldEvent); + vitessSchema.processFieldEvent(fieldEvent); + } + + private void processVGtidEvent(VEvent event) { + LOGGER.info("Received GTID event: " + event); + } + + /** + * Get a batch of rows for the current transaction. + * + * We assume the replicator has just processed a "BEGIN" event, and now + * we're inside a transaction. We'll process all rows inside that transaction + * and turn them into RowMap objects. + * + * We do this because we want to attach all rows within + * the transaction the same transaction id value (xid, which we generate + * ourselves since VStream + * does not expose underlying transaction ids to the consumer). + * + * @return A RowMapBuffer of rows; either in-memory or on disk. + */ + private RowMapBuffer getTransactionRows(VEvent beginEvent) throws Exception { + RowMapBuffer buffer = new RowMapBuffer(MAX_TX_ELEMENTS, this.bufferMemoryUsage); + + // Since transactions in VStream do not have an XID value, we generate one + long xid = System.currentTimeMillis() * 1000 + Math.abs(beginEvent.hashCode()) % 1000; + LOGGER.debug("Generated transaction id: {}", xid); + buffer.setXid(xid); + + while (true) { + final VEvent event = pollEvent(); + if (event == null) { + continue; + } + + final VEventType eventType = event.getType(); + + if (eventType == VEventType.COMMIT) { + LOGGER.debug("Received COMMIT event"); + if (!buffer.isEmpty()) { + buffer.getLast().setTXCommit(); + long timeSpent = buffer.getLast().getTimestampMillis() - beginEvent.getTimestamp(); + transactionExecutionTime.update(timeSpent); + transactionRowCount.update(buffer.size()); + } + return buffer; + } + + if (eventType == VEventType.ROW) { + List eventRows = rowEventToMaps(event, xid); + for (RowMap r : eventRows) { + // if (shouldOutputRowMap(table.getDatabase(), table.getName(), r, filter)) { + buffer.add(r); + } + continue; + } + + // All other events are service events delivering the schema, GTID values, etc. + processServiceEvent(event); + } + } + + private void processServiceEvent(VEvent event) { + final VEventType eventType = event.getType(); + switch (eventType) { + case FIELD: + processFieldEvent(event); + break; + + case HEARTBEAT: + LOGGER.debug("Received heartbeat from vtgate: {}", event); + break; + + case VGTID: + processVGtidEvent(event); + break; + + case ROW: + case BEGIN: + case COMMIT: + LOGGER.error("Unexpected event outside of a transaction, skipping: {}", event); + break; + + default: + LOGGER.warn("Unexpected service event: {}", event); + break; + } + + } + + private List rowEventToMaps(VEvent event, long xid) { + Long timestampMillis = event.getCurrentTime(); + RowEvent rowEvent = event.getRowEvent(); + String qualifiedTableName = rowEvent.getTableName(); + + List rowMaps = new ArrayList<>(rowEvent.getRowChangesCount()); + for (RowChange rowChange : rowEvent.getRowChangesList()) { + String changeType = rowChangeToMaxwellType(rowChange); + final VitessTable table = resolveTable(qualifiedTableName); + if (!filter.includes(table.getSchemaName(), table.getTableName())) { + LOGGER.debug("Filtering out event for table {}.{}", table.getSchemaName(), table.getTableName()); + continue; + } + List columns = table.resolveColumns(rowChange); + + RowMap rowMap = new RowMap( + changeType, + table.getSchemaName(), + table.getTableName(), + timestampMillis, + table.getPkColumns(), + null, null, null); + + rowMap.setXid(xid); + + for (ReplicationMessageColumn column : columns) { + rowMap.putData(column.getName(), column.getValue()); + } + + rowMaps.add(rowMap); + } + + return rowMaps; + } + + private String rowChangeToMaxwellType(RowChange change) { + if (change.hasAfter() && !change.hasBefore()) { + return INSERT_TYPE; + } + if (change.hasAfter() && change.hasBefore()) { + return UPDATE_TYPE; + } + if (!change.hasAfter() && change.hasBefore()) { + return DELETE_TYPE; + } + + throw new RuntimeException("Invalid row change: " + change); + } + + private VitessTable resolveTable(String qualifiedTableName) { + VitessTable table = vitessSchema.getTable(qualifiedTableName); + if (table == null) { + throw new RuntimeException("Table not found in the schema: " + qualifiedTableName); + } + return table; + } + + @Override + public Long getLastHeartbeatRead() { + // TODO Auto-generated method stub + return null; + } + + @Override + public Schema getSchema() throws SchemaStoreException { + // TODO Auto-generated method stub + return null; + } + + @Override + public Long getSchemaId() throws SchemaStoreException { + // TODO Auto-generated method stub + return null; + } + + @Override + public void stopAtHeartbeat(long heartbeat) { + // TODO Auto-generated method stub + } + + private static ManagedChannel newChannel(String vtgateHost, int vtgatePort, int maxInboundMessageSize) { + return ManagedChannelBuilder + .forAddress(vtgateHost, vtgatePort) + .usePlaintext() + .maxInboundMessageSize(maxInboundMessageSize) + .keepAliveTime(KEEPALIVE_INTERVAL_SECONDS, TimeUnit.SECONDS) + .build(); + } } diff --git a/src/main/java/com/zendesk/maxwell/replication/vitess/ColumnMetaData.java b/src/main/java/com/zendesk/maxwell/replication/vitess/ColumnMetaData.java index b9f293166..189370d31 100644 --- a/src/main/java/com/zendesk/maxwell/replication/vitess/ColumnMetaData.java +++ b/src/main/java/com/zendesk/maxwell/replication/vitess/ColumnMetaData.java @@ -11,31 +11,31 @@ * a {@link VitessTable}. */ public class ColumnMetaData { - private final String columnName; - private final VitessType vitessType; - private final boolean optional; - private final KeyMetaData keyMetaData; + private final String columnName; + private final VitessType vitessType; + private final boolean optional; + private final KeyMetaData keyMetaData; - public ColumnMetaData(String columnName, VitessType vitessType, boolean optional, KeyMetaData keyMetaData) { - this.columnName = columnName; - this.vitessType = vitessType; - this.keyMetaData = keyMetaData; - this.optional = optional; - } + public ColumnMetaData(String columnName, VitessType vitessType, boolean optional, KeyMetaData keyMetaData) { + this.columnName = columnName; + this.vitessType = vitessType; + this.keyMetaData = keyMetaData; + this.optional = optional; + } - public String getColumnName() { - return columnName; - } + public String getColumnName() { + return columnName; + } - public VitessType getVitessType() { - return vitessType; - } + public VitessType getVitessType() { + return vitessType; + } - public boolean isOptional() { - return optional; - } + public boolean isOptional() { + return optional; + } - public KeyMetaData getKeyMetaData() { - return keyMetaData; - } + public KeyMetaData getKeyMetaData() { + return keyMetaData; + } } diff --git a/src/main/java/com/zendesk/maxwell/replication/vitess/KeyMetaData.java b/src/main/java/com/zendesk/maxwell/replication/vitess/KeyMetaData.java index f51870c21..8c5d4606c 100644 --- a/src/main/java/com/zendesk/maxwell/replication/vitess/KeyMetaData.java +++ b/src/main/java/com/zendesk/maxwell/replication/vitess/KeyMetaData.java @@ -10,17 +10,17 @@ * or a column is a single-column unique key */ public enum KeyMetaData { - /** - * The column is part of the primary key (including multi-column primary key) - */ - IS_KEY, - /** - * The column is single-column unique key - */ - IS_UNIQUE_KEY, - /** - * The column is not part of any key, or the column is part of multi-column - * unique key - */ - NONE + /** + * The column is part of the primary key (including multi-column primary key) + */ + IS_KEY, + /** + * The column is single-column unique key + */ + IS_UNIQUE_KEY, + /** + * The column is not part of any key, or the column is part of multi-column + * unique key + */ + NONE } diff --git a/src/main/java/com/zendesk/maxwell/replication/vitess/ReplicationMessageColumn.java b/src/main/java/com/zendesk/maxwell/replication/vitess/ReplicationMessageColumn.java index 9211b94bb..cea365303 100644 --- a/src/main/java/com/zendesk/maxwell/replication/vitess/ReplicationMessageColumn.java +++ b/src/main/java/com/zendesk/maxwell/replication/vitess/ReplicationMessageColumn.java @@ -10,60 +10,60 @@ /** Logical representation of both column type and value. */ public class ReplicationMessageColumn { - private final String columnName; - private final VitessType type; - private final byte[] rawValue; + private final String columnName; + private final VitessType type; + private final byte[] rawValue; - public ReplicationMessageColumn(String columnName, VitessType type, byte[] rawValue) { - this.columnName = columnName; - this.type = type; - this.rawValue = rawValue; - } + public ReplicationMessageColumn(String columnName, VitessType type, byte[] rawValue) { + this.columnName = columnName; + this.type = type; + this.rawValue = rawValue; + } - public String getName() { - return columnName; - } + public String getName() { + return columnName; + } - public VitessType getType() { - return type; - } + public VitessType getType() { + return type; + } - public Object getValue() { - final VitessColumnValue value = new VitessColumnValue(rawValue); + public Object getValue() { + final VitessColumnValue value = new VitessColumnValue(rawValue); - if (value.isNull()) { - return null; - } + if (value.isNull()) { + return null; + } - switch (type.getJdbcId()) { - case Types.SMALLINT: - return value.asShort(); - case Types.INTEGER: - return value.asInteger(); - case Types.BIGINT: - return value.asLong(); - case Types.BLOB: - case Types.BINARY: - return value.asBytes(); - case Types.VARCHAR: - return value.asString(); - case Types.FLOAT: - return value.asFloat(); - case Types.DOUBLE: - return value.asDouble(); - default: - break; - } + switch (type.getJdbcId()) { + case Types.SMALLINT: + return value.asShort(); + case Types.INTEGER: + return value.asInteger(); + case Types.BIGINT: + return value.asLong(); + case Types.BLOB: + case Types.BINARY: + return value.asBytes(); + case Types.VARCHAR: + return value.asString(); + case Types.FLOAT: + return value.asFloat(); + case Types.DOUBLE: + return value.asDouble(); + default: + break; + } - return value.asDefault(type); - } + return value.asDefault(type); + } - public byte[] getRawValue() { - return rawValue; - } + public byte[] getRawValue() { + return rawValue; + } - @Override - public String toString() { - return columnName + "=" + new String(rawValue, StandardCharsets.UTF_8); - } + @Override + public String toString() { + return columnName + "=" + new String(rawValue, StandardCharsets.UTF_8); + } } diff --git a/src/main/java/com/zendesk/maxwell/replication/vitess/Vgtid.java b/src/main/java/com/zendesk/maxwell/replication/vitess/Vgtid.java index 9c94e698e..f59435c5b 100644 --- a/src/main/java/com/zendesk/maxwell/replication/vitess/Vgtid.java +++ b/src/main/java/com/zendesk/maxwell/replication/vitess/Vgtid.java @@ -20,138 +20,138 @@ /** Vitess source position coordinates. */ public class Vgtid { - public static final String CURRENT_GTID = "current"; - public static final String KEYSPACE_KEY = "keyspace"; - public static final String SHARD_KEY = "shard"; - public static final String GTID_KEY = "gtid"; - - private static final ObjectMapper MAPPER = new ObjectMapper(); - - private final Binlogdata.VGtid rawVgtid; - private final List shardGtids = new ArrayList<>(); - - private Vgtid(Binlogdata.VGtid rawVgtid) { - this.rawVgtid = rawVgtid; - for (Binlogdata.ShardGtid shardGtid : rawVgtid.getShardGtidsList()) { - shardGtids.add(new ShardGtid(shardGtid.getKeyspace(), shardGtid.getShard(), shardGtid.getGtid())); - } - } - - private Vgtid(List shardGtids) { - this.shardGtids.addAll(shardGtids); - - Binlogdata.VGtid.Builder builder = Binlogdata.VGtid.newBuilder(); - for (ShardGtid shardGtid : shardGtids) { - builder.addShardGtids( - Binlogdata.ShardGtid.newBuilder() - .setKeyspace(shardGtid.getKeyspace()) - .setShard(shardGtid.getShard()) - .setGtid(shardGtid.getGtid()) - .build()); - } - this.rawVgtid = builder.build(); - } - - public static Vgtid of(String shardGtidsInJson) { - try { - List shardGtids = MAPPER.readValue(shardGtidsInJson, new TypeReference>() { - }); - return of(shardGtids); - } catch (JsonProcessingException e) { - throw new IllegalStateException(e); - } - } - - public static Vgtid of(Binlogdata.VGtid rawVgtid) { - return new Vgtid(rawVgtid); - } - - public static Vgtid of(List shardGtids) { - return new Vgtid(shardGtids); - } - - public Binlogdata.VGtid getRawVgtid() { - return rawVgtid; - } - - public List getShardGtids() { - return shardGtids; - } - - public boolean isSingleShard() { - return rawVgtid.getShardGtidsCount() == 1; - } - - @Override - public String toString() { - try { - return MAPPER.writeValueAsString(shardGtids); - } catch (JsonProcessingException e) { - throw new IllegalStateException(e); - } - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - Vgtid vgtid = (Vgtid) o; - return Objects.equals(rawVgtid, vgtid.rawVgtid) && - Objects.equals(shardGtids, vgtid.shardGtids); - } - - @Override - public int hashCode() { - return Objects.hash(rawVgtid, shardGtids); - } - - @JsonPropertyOrder({ KEYSPACE_KEY, SHARD_KEY, GTID_KEY }) - public static class ShardGtid { - private final String keyspace; - private final String shard; - private final String gtid; - - @JsonCreator - public ShardGtid(@JsonProperty(KEYSPACE_KEY) String keyspace, @JsonProperty(SHARD_KEY) String shard, - @JsonProperty(GTID_KEY) String gtid) { - this.keyspace = keyspace; - this.shard = shard; - this.gtid = gtid; - } - - public String getKeyspace() { - return keyspace; - } - - public String getShard() { - return shard; - } - - public String getGtid() { - return gtid; - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - ShardGtid shardGtid = (ShardGtid) o; - return Objects.equals(keyspace, shardGtid.keyspace) && - Objects.equals(shard, shardGtid.shard) && - Objects.equals(gtid, shardGtid.gtid); - } - - @Override - public int hashCode() { - return Objects.hash(keyspace, shard, gtid); - } - } + public static final String CURRENT_GTID = "current"; + public static final String KEYSPACE_KEY = "keyspace"; + public static final String SHARD_KEY = "shard"; + public static final String GTID_KEY = "gtid"; + + private static final ObjectMapper MAPPER = new ObjectMapper(); + + private final Binlogdata.VGtid rawVgtid; + private final List shardGtids = new ArrayList<>(); + + private Vgtid(Binlogdata.VGtid rawVgtid) { + this.rawVgtid = rawVgtid; + for (Binlogdata.ShardGtid shardGtid : rawVgtid.getShardGtidsList()) { + shardGtids.add(new ShardGtid(shardGtid.getKeyspace(), shardGtid.getShard(), shardGtid.getGtid())); + } + } + + private Vgtid(List shardGtids) { + this.shardGtids.addAll(shardGtids); + + Binlogdata.VGtid.Builder builder = Binlogdata.VGtid.newBuilder(); + for (ShardGtid shardGtid : shardGtids) { + builder.addShardGtids( + Binlogdata.ShardGtid.newBuilder() + .setKeyspace(shardGtid.getKeyspace()) + .setShard(shardGtid.getShard()) + .setGtid(shardGtid.getGtid()) + .build()); + } + this.rawVgtid = builder.build(); + } + + public static Vgtid of(String shardGtidsInJson) { + try { + List shardGtids = MAPPER.readValue(shardGtidsInJson, new TypeReference>() { + }); + return of(shardGtids); + } catch (JsonProcessingException e) { + throw new IllegalStateException(e); + } + } + + public static Vgtid of(Binlogdata.VGtid rawVgtid) { + return new Vgtid(rawVgtid); + } + + public static Vgtid of(List shardGtids) { + return new Vgtid(shardGtids); + } + + public Binlogdata.VGtid getRawVgtid() { + return rawVgtid; + } + + public List getShardGtids() { + return shardGtids; + } + + public boolean isSingleShard() { + return rawVgtid.getShardGtidsCount() == 1; + } + + @Override + public String toString() { + try { + return MAPPER.writeValueAsString(shardGtids); + } catch (JsonProcessingException e) { + throw new IllegalStateException(e); + } + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + Vgtid vgtid = (Vgtid) o; + return Objects.equals(rawVgtid, vgtid.rawVgtid) && + Objects.equals(shardGtids, vgtid.shardGtids); + } + + @Override + public int hashCode() { + return Objects.hash(rawVgtid, shardGtids); + } + + @JsonPropertyOrder({ KEYSPACE_KEY, SHARD_KEY, GTID_KEY }) + public static class ShardGtid { + private final String keyspace; + private final String shard; + private final String gtid; + + @JsonCreator + public ShardGtid(@JsonProperty(KEYSPACE_KEY) String keyspace, @JsonProperty(SHARD_KEY) String shard, + @JsonProperty(GTID_KEY) String gtid) { + this.keyspace = keyspace; + this.shard = shard; + this.gtid = gtid; + } + + public String getKeyspace() { + return keyspace; + } + + public String getShard() { + return shard; + } + + public String getGtid() { + return gtid; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + ShardGtid shardGtid = (ShardGtid) o; + return Objects.equals(keyspace, shardGtid.keyspace) && + Objects.equals(shard, shardGtid.shard) && + Objects.equals(gtid, shardGtid.gtid); + } + + @Override + public int hashCode() { + return Objects.hash(keyspace, shard, gtid); + } + } } diff --git a/src/main/java/com/zendesk/maxwell/replication/vitess/VitessColumn.java b/src/main/java/com/zendesk/maxwell/replication/vitess/VitessColumn.java index 7ceb9a6b4..9d03e6a9b 100644 --- a/src/main/java/com/zendesk/maxwell/replication/vitess/VitessColumn.java +++ b/src/main/java/com/zendesk/maxwell/replication/vitess/VitessColumn.java @@ -1,23 +1,23 @@ package com.zendesk.maxwell.replication.vitess; public class VitessColumn { - private final String name; - private final VitessType type; + private final String name; + private final VitessType type; - public VitessColumn(String name, VitessType type) { - this.name = name; - this.type = type; - } + public VitessColumn(String name, VitessType type) { + this.name = name; + this.type = type; + } - public String getName() { - return name; - } + public String getName() { + return name; + } - public VitessType getType() { - return type; - } + public VitessType getType() { + return type; + } - public String toString() { - return "Column [name=" + name + ", type=" + type + "]"; - } + public String toString() { + return "Column [name=" + name + ", type=" + type + "]"; + } } diff --git a/src/main/java/com/zendesk/maxwell/replication/vitess/VitessColumnValue.java b/src/main/java/com/zendesk/maxwell/replication/vitess/VitessColumnValue.java index 1a740fd6c..83b18ba22 100644 --- a/src/main/java/com/zendesk/maxwell/replication/vitess/VitessColumnValue.java +++ b/src/main/java/com/zendesk/maxwell/replication/vitess/VitessColumnValue.java @@ -15,62 +15,62 @@ * value. */ public class VitessColumnValue { - private static final Logger LOGGER = LoggerFactory.getLogger(VitessColumnValue.class); + private static final Logger LOGGER = LoggerFactory.getLogger(VitessColumnValue.class); - private final byte[] value; + private final byte[] value; - public VitessColumnValue(byte[] value) { - this.value = value; - } + public VitessColumnValue(byte[] value) { + this.value = value; + } - public byte[] getRawValue() { - return value; - } + public byte[] getRawValue() { + return value; + } - public boolean isNull() { - return value == null; - } + public boolean isNull() { + return value == null; + } - public byte[] asBytes() { - return value; - } + public byte[] asBytes() { + return value; + } - /** - * Convert raw bytes value to string using UTF-8 encoding. - * - * This is *enforced* for VARCHAR and CHAR types, and is *required* for other - * non-bytes types (numeric, - * timestamp, etc.). For bytes (BLOB, BINARY, etc.) types, the asBytes() should - * be used. - * - * @return the UTF-8 string - */ - public String asString() { - return new String(value, StandardCharsets.UTF_8); - } + /** + * Convert raw bytes value to string using UTF-8 encoding. + * + * This is *enforced* for VARCHAR and CHAR types, and is *required* for other + * non-bytes types (numeric, + * timestamp, etc.). For bytes (BLOB, BINARY, etc.) types, the asBytes() should + * be used. + * + * @return the UTF-8 string + */ + public String asString() { + return new String(value, StandardCharsets.UTF_8); + } - public Integer asInteger() { - return Integer.valueOf(asString()); - } + public Integer asInteger() { + return Integer.valueOf(asString()); + } - public Short asShort() { - return Short.valueOf(asString()); - } + public Short asShort() { + return Short.valueOf(asString()); + } - public Long asLong() { - return Long.valueOf(asString()); - } + public Long asLong() { + return Long.valueOf(asString()); + } - public Float asFloat() { - return Float.valueOf(asString()); - } + public Float asFloat() { + return Float.valueOf(asString()); + } - public Double asDouble() { - return Double.valueOf(asString()); - } + public Double asDouble() { + return Double.valueOf(asString()); + } - public Object asDefault(VitessType vitessType) { - LOGGER.warn("processing unknown column type {} as string", vitessType); - return asString(); - } + public Object asDefault(VitessType vitessType) { + LOGGER.warn("processing unknown column type {} as string", vitessType); + return asString(); + } } diff --git a/src/main/java/com/zendesk/maxwell/replication/vitess/VitessSchema.java b/src/main/java/com/zendesk/maxwell/replication/vitess/VitessSchema.java index 421f4a755..219a7a9f3 100644 --- a/src/main/java/com/zendesk/maxwell/replication/vitess/VitessSchema.java +++ b/src/main/java/com/zendesk/maxwell/replication/vitess/VitessSchema.java @@ -14,112 +14,112 @@ // An in-memory representation of a Vitess schema. public class VitessSchema { - private static final Logger LOGGER = LoggerFactory.getLogger(VitessSchema.class); - - // See all flags: - // https://dev.mysql.com/doc/dev/mysql-server/8.0.12/group__group__cs__column__definition__flags.html - private static final int NOT_NULL_FLAG = 1; - private static final int PRI_KEY_FLAG = 1 << 1; - private static final int UNIQUE_KEY_FLAG = 1 << 2; - - private final Map tables = new HashMap<>(); - - public void addTable(VitessTable table) { - String tableName = table.getQualifiedTableName(); - if (tables.containsKey(tableName)) { - LOGGER.info("Schema change detected for: {}", table); - } else { - LOGGER.info("Table schema received for: {}", table); - } - tables.put(tableName, table); - } - - public VitessTable getTable(String name) { - return tables.get(name); - } - - public void processFieldEvent(FieldEvent event) { - if (event == null) { - throw new RuntimeException(String.format("fieldEvent is expected from {}", event)); - } - - String qualifiedTableName = event.getTableName(); - String[] schemaTableTuple = qualifiedTableName.split("\\."); - if (schemaTableTuple.length != 2) { - throw new RuntimeException( - String.format( - "Handling FIELD VEvent. schemaTableTuple should have schema name and table name but has size {}. {} is skipped", - schemaTableTuple.length, - event)); - } - - LOGGER.debug("Handling FIELD VEvent: {}", event); - String schemaName = schemaTableTuple[0]; - String tableName = schemaTableTuple[1]; - int columnCount = event.getFieldsCount(); - - List columns = new ArrayList<>(columnCount); - for (short i = 0; i < columnCount; ++i) { - Field field = event.getFields(i); - String columnName = validateColumnName(field.getName(), schemaName, tableName); - - VitessType vitessType = VitessType.resolve(field); - if (vitessType.getJdbcId() == Types.OTHER) { - LOGGER.error("Cannot resolve JDBC type from VStream field {}", field); - } - - KeyMetaData keyMetaData = KeyMetaData.NONE; - if ((field.getFlags() & PRI_KEY_FLAG) != 0) { - keyMetaData = KeyMetaData.IS_KEY; - } else if ((field.getFlags() & UNIQUE_KEY_FLAG) != 0) { - keyMetaData = KeyMetaData.IS_UNIQUE_KEY; - } - boolean optional = (field.getFlags() & NOT_NULL_FLAG) == 0; - - columns.add(new ColumnMetaData(columnName, vitessType, optional, keyMetaData)); - } - - VitessTable table = createTable(schemaName, tableName, columns); - addTable(table); - } - - private static String validateColumnName(String columnName, String schemaName, String tableName) { - int length = columnName.length(); - if (length == 0) { - throw new IllegalArgumentException( - String.format("Empty column name from schema: %s, table: %s", schemaName, tableName)); - } - - // Vitess VStreamer schema reloading transient bug could cause column names to - // be anonymized to @1, @2, etc - // We want to fail in this case instead of sending the corrupted row events with - // @1, @2 as column names. - char first = columnName.charAt(0); - if (first == '@') { - throw new IllegalArgumentException( - String.format( - "Illegal prefix '@' for column: %s, from schema: %s, table: %s", - columnName, - schemaName, - tableName)); - } - - return columnName; - } - - private VitessTable createTable(String schemaName, String tableName, List columnsMetaData) { - List columns = new ArrayList<>(columnsMetaData.size()); - List pkColumns = new ArrayList<>(); - - for (ColumnMetaData columnMetaData : columnsMetaData) { - VitessColumn column = new VitessColumn(columnMetaData.getColumnName(), columnMetaData.getVitessType()); - columns.add(column); - - if (columnMetaData.getKeyMetaData() == KeyMetaData.IS_KEY) { - pkColumns.add(columnMetaData.getColumnName()); - } - } - - return new VitessTable(schemaName, tableName, columns, pkColumns); - } + private static final Logger LOGGER = LoggerFactory.getLogger(VitessSchema.class); + + // See all flags: + // https://dev.mysql.com/doc/dev/mysql-server/8.0.12/group__group__cs__column__definition__flags.html + private static final int NOT_NULL_FLAG = 1; + private static final int PRI_KEY_FLAG = 1 << 1; + private static final int UNIQUE_KEY_FLAG = 1 << 2; + + private final Map tables = new HashMap<>(); + + public void addTable(VitessTable table) { + String tableName = table.getQualifiedTableName(); + if (tables.containsKey(tableName)) { + LOGGER.info("Schema change detected for: {}", table); + } else { + LOGGER.info("Table schema received for: {}", table); + } + tables.put(tableName, table); + } + + public VitessTable getTable(String name) { + return tables.get(name); + } + + public void processFieldEvent(FieldEvent event) { + if (event == null) { + throw new RuntimeException(String.format("fieldEvent is expected from {}", event)); + } + + String qualifiedTableName = event.getTableName(); + String[] schemaTableTuple = qualifiedTableName.split("\\."); + if (schemaTableTuple.length != 2) { + throw new RuntimeException( + String.format( + "Handling FIELD VEvent. schemaTableTuple should have schema name and table name but has size {}. {} is skipped", + schemaTableTuple.length, + event)); + } + + LOGGER.debug("Handling FIELD VEvent: {}", event); + String schemaName = schemaTableTuple[0]; + String tableName = schemaTableTuple[1]; + int columnCount = event.getFieldsCount(); + + List columns = new ArrayList<>(columnCount); + for (short i = 0; i < columnCount; ++i) { + Field field = event.getFields(i); + String columnName = validateColumnName(field.getName(), schemaName, tableName); + + VitessType vitessType = VitessType.resolve(field); + if (vitessType.getJdbcId() == Types.OTHER) { + LOGGER.error("Cannot resolve JDBC type from VStream field {}", field); + } + + KeyMetaData keyMetaData = KeyMetaData.NONE; + if ((field.getFlags() & PRI_KEY_FLAG) != 0) { + keyMetaData = KeyMetaData.IS_KEY; + } else if ((field.getFlags() & UNIQUE_KEY_FLAG) != 0) { + keyMetaData = KeyMetaData.IS_UNIQUE_KEY; + } + boolean optional = (field.getFlags() & NOT_NULL_FLAG) == 0; + + columns.add(new ColumnMetaData(columnName, vitessType, optional, keyMetaData)); + } + + VitessTable table = createTable(schemaName, tableName, columns); + addTable(table); + } + + private static String validateColumnName(String columnName, String schemaName, String tableName) { + int length = columnName.length(); + if (length == 0) { + throw new IllegalArgumentException( + String.format("Empty column name from schema: %s, table: %s", schemaName, tableName)); + } + + // Vitess VStreamer schema reloading transient bug could cause column names to + // be anonymized to @1, @2, etc + // We want to fail in this case instead of sending the corrupted row events with + // @1, @2 as column names. + char first = columnName.charAt(0); + if (first == '@') { + throw new IllegalArgumentException( + String.format( + "Illegal prefix '@' for column: %s, from schema: %s, table: %s", + columnName, + schemaName, + tableName)); + } + + return columnName; + } + + private VitessTable createTable(String schemaName, String tableName, List columnsMetaData) { + List columns = new ArrayList<>(columnsMetaData.size()); + List pkColumns = new ArrayList<>(); + + for (ColumnMetaData columnMetaData : columnsMetaData) { + VitessColumn column = new VitessColumn(columnMetaData.getColumnName(), columnMetaData.getVitessType()); + columns.add(column); + + if (columnMetaData.getKeyMetaData() == KeyMetaData.IS_KEY) { + pkColumns.add(columnMetaData.getColumnName()); + } + } + + return new VitessTable(schemaName, tableName, columns, pkColumns); + } } diff --git a/src/main/java/com/zendesk/maxwell/replication/vitess/VitessTable.java b/src/main/java/com/zendesk/maxwell/replication/vitess/VitessTable.java index e9ed928b0..abb3db809 100644 --- a/src/main/java/com/zendesk/maxwell/replication/vitess/VitessTable.java +++ b/src/main/java/com/zendesk/maxwell/replication/vitess/VitessTable.java @@ -9,81 +9,81 @@ import io.vitess.proto.Query.Row; public class VitessTable { - private final String schemaName; - private final String tableName; - private final List columns; - private final List pkColumns; + private final String schemaName; + private final String tableName; + private final List columns; + private final List pkColumns; - public VitessTable(String schemaName, String tableName, List columns, List pkColumns) { - this.schemaName = schemaName; - this.tableName = tableName; - this.columns = columns; - this.pkColumns = pkColumns; - } + public VitessTable(String schemaName, String tableName, List columns, List pkColumns) { + this.schemaName = schemaName; + this.tableName = tableName; + this.columns = columns; + this.pkColumns = pkColumns; + } - public String getSchemaName() { - return schemaName; - } + public String getSchemaName() { + return schemaName; + } - public String getTableName() { - return tableName; - } + public String getTableName() { + return tableName; + } - public String getQualifiedTableName() { - return schemaName + "." + tableName; - } + public String getQualifiedTableName() { + return schemaName + "." + tableName; + } - public List getColumns() { - return columns; - } + public List getColumns() { + return columns; + } - public List getPkColumns() { - return pkColumns; - } + public List getPkColumns() { + return pkColumns; + } - public String toString() { - return "Table [schemaName=" + schemaName - + ", tableName=" + tableName - + ", columns=" + columns - + ", pkColumns=" + pkColumns - + "]"; - } + public String toString() { + return "Table [schemaName=" + schemaName + + ", tableName=" + tableName + + ", columns=" + columns + + ", pkColumns=" + pkColumns + + "]"; + } - /** - * Resolve the vEvent data to a list of replication message columns (with - * values). - */ - public List resolveColumns(RowChange rowChange) { - Row row = rowChange.hasAfter() ? rowChange.getAfter() : rowChange.getBefore(); - int changedColumnsCnt = row.getLengthsCount(); - if (columns.size() != changedColumnsCnt) { - throw new IllegalStateException( - String.format( - "The number of columns in the ROW event {} is different from the in-memory table schema {}.", - row, - this)); - } + /** + * Resolve the vEvent data to a list of replication message columns (with + * values). + */ + public List resolveColumns(RowChange rowChange) { + Row row = rowChange.hasAfter() ? rowChange.getAfter() : rowChange.getBefore(); + int changedColumnsCnt = row.getLengthsCount(); + if (columns.size() != changedColumnsCnt) { + throw new IllegalStateException( + String.format( + "The number of columns in the ROW event {} is different from the in-memory table schema {}.", + row, + this)); + } - ByteString rawValues = row.getValues(); - int rawValueIndex = 0; - List eventColumns = new ArrayList<>(changedColumnsCnt); - for (short i = 0; i < changedColumnsCnt; i++) { - final VitessColumn columnDefinition = columns.get(i); - final String columnName = columnDefinition.getName(); - final VitessType vitessType = columnDefinition.getType(); + ByteString rawValues = row.getValues(); + int rawValueIndex = 0; + List eventColumns = new ArrayList<>(changedColumnsCnt); + for (short i = 0; i < changedColumnsCnt; i++) { + final VitessColumn columnDefinition = columns.get(i); + final String columnName = columnDefinition.getName(); + final VitessType vitessType = columnDefinition.getType(); - final int rawValueLength = (int) row.getLengths(i); - final byte[] rawValue = rawValueLength == -1 - ? null - : rawValues.substring(rawValueIndex, rawValueIndex + rawValueLength).toByteArray(); - if (rawValueLength != -1) { - // no update to rawValueIndex when no value in the rawValue - rawValueIndex += rawValueLength; - } + final int rawValueLength = (int) row.getLengths(i); + final byte[] rawValue = rawValueLength == -1 + ? null + : rawValues.substring(rawValueIndex, rawValueIndex + rawValueLength).toByteArray(); + if (rawValueLength != -1) { + // no update to rawValueIndex when no value in the rawValue + rawValueIndex += rawValueLength; + } - final ReplicationMessageColumn column = new ReplicationMessageColumn(columnName, vitessType, rawValue); - eventColumns.add(column); - } - return eventColumns; - } + final ReplicationMessageColumn column = new ReplicationMessageColumn(columnName, vitessType, rawValue); + eventColumns.add(column); + } + return eventColumns; + } } diff --git a/src/main/java/com/zendesk/maxwell/replication/vitess/VitessType.java b/src/main/java/com/zendesk/maxwell/replication/vitess/VitessType.java index 68cd4e106..289241352 100644 --- a/src/main/java/com/zendesk/maxwell/replication/vitess/VitessType.java +++ b/src/main/java/com/zendesk/maxwell/replication/vitess/VitessType.java @@ -16,148 +16,149 @@ /** The Vitess table column type */ public class VitessType { - // name of the column type - private final String name; - // enum of column jdbc type - private final int jdbcId; - // permitted enum values - private final List enumValues; - - public VitessType(String name, int jdbcId) { - this(name, jdbcId, Collections.emptyList()); - } - - public VitessType(String name, int jdbcId, List enumValues) { - this.name = name; - this.jdbcId = jdbcId; - this.enumValues = Collections.unmodifiableList(enumValues); - } - - public String getName() { - return name; - } - - public int getJdbcId() { - return jdbcId; - } - - public List getEnumValues() { - return enumValues; - } - - public boolean isEnum() { - return !enumValues.isEmpty(); - } - - @Override - public String toString() { - return "VitessType{" + - "name='" + name + '\'' + - ", jdbcId=" + jdbcId + - ", enumValues=" + enumValues + - '}'; - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - VitessType that = (VitessType) o; - return jdbcId == that.jdbcId && name.equals(that.name) && Objects.equals(enumValues, that.enumValues); - } - - @Override - public int hashCode() { - return Objects.hash(name, jdbcId, enumValues); - } - - // Resolve JDBC type from vstream FIELD event - public static VitessType resolve(Field field) { - final String type = field.getType().name(); - switch (type) { - case "INT8": - case "UINT8": - case "INT16": - return new VitessType(type, Types.SMALLINT); - case "UINT16": - case "INT24": - case "UINT24": - case "INT32": - return new VitessType(type, Types.INTEGER); - case "ENUM": - return new VitessType(type, Types.INTEGER, resolveEnumAndSetValues(field.getColumnType())); - case "SET": - return new VitessType(type, Types.BIGINT, resolveEnumAndSetValues(field.getColumnType())); - case "UINT32": - case "INT64": - return new VitessType(type, Types.BIGINT); - case "BLOB": - return new VitessType(type, Types.BLOB); - case "VARBINARY": - case "BINARY": - return new VitessType(type, Types.BINARY); - case "UINT64": - case "VARCHAR": - case "CHAR": - case "TEXT": - case "JSON": - case "DECIMAL": - case "TIME": - case "DATE": - case "DATETIME": - case "TIMESTAMP": - case "YEAR": - return new VitessType(type, Types.VARCHAR); - case "FLOAT32": - return new VitessType(type, Types.FLOAT); - case "FLOAT64": - return new VitessType(type, Types.DOUBLE); - default: - return new VitessType(type, Types.OTHER); - } - } - - /** - * Resolve the list of permitted Enum or Set values from the Enum or Set - * Definition - * - * @param definition the Enum or Set column definition from the MySQL table. - * E.g. "enum('m','l','xl')" or "set('a','b','c')" - * @return The list of permitted Enum values or Set values - */ - private static List resolveEnumAndSetValues(String definition) { - List values = new ArrayList<>(); - if (definition == null || definition.length() == 0) { - return values; - } - - StringBuilder sb = new StringBuilder(); - boolean startCollecting = false; - char[] chars = definition.toCharArray(); - for (int i = 0; i < chars.length; i++) { - if (chars[i] == '\'') { - if (chars[i + 1] != '\'') { - if (startCollecting) { - // end of the Enum/Set value, add the Enum/Set value to the result list - values.add(sb.toString()); - sb.setLength(0); - } - startCollecting = !startCollecting; - } else { - sb.append("'"); - // In MySQL, the single quote in the Enum/Set definition "a'b" is escaped and - // becomes "a''b". - // Skip the second single-quote - i++; - } - } else if (startCollecting) { - sb.append(chars[i]); - } - } - return values; - } + // name of the column type + private final String name; + // enum of column jdbc type + private final int jdbcId; + // permitted enum values + private final List enumValues; + + public VitessType(String name, int jdbcId) { + this(name, jdbcId, Collections.emptyList()); + } + + public VitessType(String name, int jdbcId, List enumValues) { + this.name = name; + this.jdbcId = jdbcId; + this.enumValues = Collections.unmodifiableList(enumValues); + } + + public String getName() { + return name; + } + + public int getJdbcId() { + return jdbcId; + } + + public List getEnumValues() { + return enumValues; + } + + public boolean isEnum() { + return !enumValues.isEmpty(); + } + + @Override + public String toString() { + return "VitessType{" + + "name='" + name + '\'' + + ", jdbcId=" + jdbcId + + ", enumValues=" + enumValues + + '}'; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + VitessType that = (VitessType) o; + return jdbcId == that.jdbcId && name.equals(that.name) && Objects.equals(enumValues, that.enumValues); + } + + @Override + public int hashCode() { + return Objects.hash(name, jdbcId, enumValues); + } + + // Resolve JDBC type from vstream FIELD event + public static VitessType resolve(Field field) { + final String type = field.getType().name(); + switch (type) { + case "INT8": + case "UINT8": + case "INT16": + return new VitessType(type, Types.SMALLINT); + case "UINT16": + case "INT24": + case "UINT24": + case "INT32": + return new VitessType(type, Types.INTEGER); + case "ENUM": + return new VitessType(type, Types.INTEGER, resolveEnumAndSetValues(field.getColumnType())); + case "SET": + return new VitessType(type, Types.BIGINT, resolveEnumAndSetValues(field.getColumnType())); + case "UINT32": + case "INT64": + return new VitessType(type, Types.BIGINT); + case "BLOB": + return new VitessType(type, Types.BLOB); + case "VARBINARY": + case "BINARY": + return new VitessType(type, Types.BINARY); + case "UINT64": + case "VARCHAR": + case "CHAR": + case "TEXT": + case "JSON": + case "DECIMAL": + case "TIME": + case "DATE": + case "DATETIME": + case "TIMESTAMP": + case "YEAR": + return new VitessType(type, Types.VARCHAR); + case "FLOAT32": + return new VitessType(type, Types.FLOAT); + case "FLOAT64": + return new VitessType(type, Types.DOUBLE); + default: + return new VitessType(type, Types.OTHER); + } + } + + /** + * Resolve the list of permitted Enum or Set values from the Enum or Set + * Definition + * + * @param definition + * the Enum or Set column definition from the MySQL table. + * E.g. "enum('m','l','xl')" or "set('a','b','c')" + * @return The list of permitted Enum values or Set values + */ + private static List resolveEnumAndSetValues(String definition) { + List values = new ArrayList<>(); + if (definition == null || definition.length() == 0) { + return values; + } + + StringBuilder sb = new StringBuilder(); + boolean startCollecting = false; + char[] chars = definition.toCharArray(); + for (int i = 0; i < chars.length; i++) { + if (chars[i] == '\'') { + if (chars[i + 1] != '\'') { + if (startCollecting) { + // end of the Enum/Set value, add the Enum/Set value to the result list + values.add(sb.toString()); + sb.setLength(0); + } + startCollecting = !startCollecting; + } else { + sb.append("'"); + // In MySQL, the single quote in the Enum/Set definition "a'b" is escaped and + // becomes "a''b". + // Skip the second single-quote + i++; + } + } else if (startCollecting) { + sb.append(chars[i]); + } + } + return values; + } } From 361f080ff1af013e37c3e9b6c9fa914ffe70db64 Mon Sep 17 00:00:00 2001 From: Oleksiy Kovyrin Date: Mon, 21 Nov 2022 16:06:57 -0500 Subject: [PATCH 03/32] More space->tabs conversion --- .../java/com/zendesk/maxwell/Maxwell.java | 107 +++++++++--------- .../com/zendesk/maxwell/MaxwellConfig.java | 2 +- .../com/zendesk/maxwell/MaxwellContext.java | 24 +++- .../maxwell/monitoring/MaxwellMetrics.java | 2 +- 4 files changed, 73 insertions(+), 62 deletions(-) diff --git a/src/main/java/com/zendesk/maxwell/Maxwell.java b/src/main/java/com/zendesk/maxwell/Maxwell.java index 61575ab0e..7628d8243 100644 --- a/src/main/java/com/zendesk/maxwell/Maxwell.java +++ b/src/main/java/com/zendesk/maxwell/Maxwell.java @@ -252,60 +252,59 @@ private void startInner() throws Exception { AbstractProducer producer = this.context.getProducer(); - if (config.vitessEnabled) { - this.replicator = new VStreamReplicator( - config.vitessConfig, - producer, - context.getMetrics(), - context.getFilter(), - config.bufferMemoryUsage, - config.binlogEventQueueSize - ); - } else { - try(Connection connection = context.getReplicationConnection()) { - MaxwellMysqlStatus.ensureReplicationMysqlState(connection); - - if (config.gtidMode) { - MaxwellMysqlStatus.ensureGtidMysqlState(connection); - } - } - - Position initPosition = getInitialPosition(); - logBanner(producer, initPosition); - this.context.setPosition(initPosition); - - MysqlSchemaStore mysqlSchemaStore = new MysqlSchemaStore(this.context, initPosition); - BootstrapController bootstrapController = this.context.getBootstrapController(mysqlSchemaStore.getSchemaID()); - - this.context.startSchemaCompactor(); - - if (config.recaptureSchema) { - mysqlSchemaStore.captureAndSaveSchema(); - } - - mysqlSchemaStore.getSchema(); // trigger schema to load / capture before we start the replicator. - - this.replicator = new BinlogConnectorReplicator( - mysqlSchemaStore, - producer, - bootstrapController, - config.replicationMysql, - config.replicaServerID, - config.databaseName, - context.getMetrics(), - initPosition, - false, - config.clientID, - context.getHeartbeatNotifier(), - config.scripting, - context.getFilter(), - context.getConfig().getIgnoreMissingSchema(), - config.outputConfig, - config.bufferMemoryUsage, - config.replicationReconnectionRetries, - config.binlogEventQueueSize - ); - } + if (config.vitessEnabled) { + this.replicator = new VStreamReplicator( + config.vitessConfig, + producer, + context.getMetrics(), + context.getFilter(), + config.bufferMemoryUsage, + config.binlogEventQueueSize); + } else { + try (Connection connection = context.getReplicationConnection()) { + MaxwellMysqlStatus.ensureReplicationMysqlState(connection); + + if (config.gtidMode) { + MaxwellMysqlStatus.ensureGtidMysqlState(connection); + } + } + + Position initPosition = getInitialPosition(); + logBanner(producer, initPosition); + this.context.setPosition(initPosition); + + MysqlSchemaStore mysqlSchemaStore = new MysqlSchemaStore(this.context, initPosition); + BootstrapController bootstrapController = this.context + .getBootstrapController(mysqlSchemaStore.getSchemaID()); + + this.context.startSchemaCompactor(); + + if (config.recaptureSchema) { + mysqlSchemaStore.captureAndSaveSchema(); + } + + mysqlSchemaStore.getSchema(); // trigger schema to load / capture before we start the replicator. + + this.replicator = new BinlogConnectorReplicator( + mysqlSchemaStore, + producer, + bootstrapController, + config.replicationMysql, + config.replicaServerID, + config.databaseName, + context.getMetrics(), + initPosition, + false, + config.clientID, + context.getHeartbeatNotifier(), + config.scripting, + context.getFilter(), + context.getConfig().getIgnoreMissingSchema(), + config.outputConfig, + config.bufferMemoryUsage, + config.replicationReconnectionRetries, + config.binlogEventQueueSize); + } context.setReplicator(replicator); this.context.start(); diff --git a/src/main/java/com/zendesk/maxwell/MaxwellConfig.java b/src/main/java/com/zendesk/maxwell/MaxwellConfig.java index 1750a28f6..fa9574c96 100644 --- a/src/main/java/com/zendesk/maxwell/MaxwellConfig.java +++ b/src/main/java/com/zendesk/maxwell/MaxwellConfig.java @@ -89,7 +89,7 @@ public class MaxwellConfig extends AbstractConfig { /** * Vitess (vtgate) connection config */ - public MaxwellVitessConfig vitessConfig; + public MaxwellVitessConfig vitessConfig; /** * Name of database in which to store maxwell data (default `maxwell`) diff --git a/src/main/java/com/zendesk/maxwell/MaxwellContext.java b/src/main/java/com/zendesk/maxwell/MaxwellContext.java index eeeec997c..57ddc0b56 100644 --- a/src/main/java/com/zendesk/maxwell/MaxwellContext.java +++ b/src/main/java/com/zendesk/maxwell/MaxwellContext.java @@ -129,11 +129,23 @@ public MaxwellContext(MaxwellConfig config) throws SQLException, URISyntaxExcept this.initialPosition = this.config.initPosition; if ( this.config.replayMode ) { - this.positionStore = new ReadOnlyMysqlPositionStore(this.getMaxwellConnectionPool(), this.getServerID(), this.config.clientID, config.gtidMode); + this.positionStore = new ReadOnlyMysqlPositionStore( + this.getMaxwellConnectionPool(), + this.getServerID(), + this.config.clientID, + config.gtidMode); } else if (this.config.vitessEnabled) { - this.positionStore = new VitessPositionStore(this.getMaxwellConnectionPool(), this.getServerID(), this.config.clientID, config.gtidMode); - } else { - this.positionStore = new MysqlPositionStore(this.getMaxwellConnectionPool(), this.getServerID(), this.config.clientID, config.gtidMode); + this.positionStore = new VitessPositionStore( + this.getMaxwellConnectionPool(), + this.getServerID(), + this.config.clientID, + config.gtidMode); + } else { + this.positionStore = new MysqlPositionStore( + this.getMaxwellConnectionPool(), + this.getServerID(), + this.config.clientID, + config.gtidMode); } this.heartbeatNotifier = new HeartbeatNotifier(); @@ -468,8 +480,8 @@ public Long getServerID() throws SQLException { return this.serverID; try ( Connection c = getReplicationConnection(); - Statement s = c.createStatement(); - ResultSet rs = s.executeQuery("SELECT @@server_id as server_id") ) { + Statement s = c.createStatement(); + ResultSet rs = s.executeQuery("SELECT @@server_id as server_id")) { if ( !rs.next() ) { throw new RuntimeException("Could not retrieve server_id!"); } diff --git a/src/main/java/com/zendesk/maxwell/monitoring/MaxwellMetrics.java b/src/main/java/com/zendesk/maxwell/monitoring/MaxwellMetrics.java index c7c97177b..9adc374b6 100644 --- a/src/main/java/com/zendesk/maxwell/monitoring/MaxwellMetrics.java +++ b/src/main/java/com/zendesk/maxwell/monitoring/MaxwellMetrics.java @@ -126,7 +126,7 @@ private void setup(MaxwellConfig config) { } if (config.metricsReportingType.contains(reportingTypeStackdriver)) { - throw new RuntimeException("Stackdriver metrics reporting is not supported for now"); + throw new RuntimeException("Stackdriver metrics reporting is not supported for now"); // io.opencensus.metrics.Metrics.getExportComponent().getMetricProducerManager().add( // new io.opencensus.contrib.dropwizard.DropWizardMetrics( // Collections.singletonList(this.registry))); From eca039545005af507f6dca6a9a863e0b09d07b3d Mon Sep 17 00:00:00 2001 From: Oleksiy Kovyrin Date: Tue, 22 Nov 2022 12:48:09 -0500 Subject: [PATCH 04/32] Fix a tab vs space issue --- .../java/com/zendesk/maxwell/Maxwell.java | 50 +++++++++---------- 1 file changed, 25 insertions(+), 25 deletions(-) diff --git a/src/main/java/com/zendesk/maxwell/Maxwell.java b/src/main/java/com/zendesk/maxwell/Maxwell.java index 7628d8243..86cc99706 100644 --- a/src/main/java/com/zendesk/maxwell/Maxwell.java +++ b/src/main/java/com/zendesk/maxwell/Maxwell.java @@ -277,34 +277,34 @@ private void startInner() throws Exception { BootstrapController bootstrapController = this.context .getBootstrapController(mysqlSchemaStore.getSchemaID()); - this.context.startSchemaCompactor(); + this.context.startSchemaCompactor(); - if (config.recaptureSchema) { - mysqlSchemaStore.captureAndSaveSchema(); - } + if (config.recaptureSchema) { + mysqlSchemaStore.captureAndSaveSchema(); + } - mysqlSchemaStore.getSchema(); // trigger schema to load / capture before we start the replicator. + mysqlSchemaStore.getSchema(); // trigger schema to load / capture before we start the replicator. - this.replicator = new BinlogConnectorReplicator( - mysqlSchemaStore, - producer, - bootstrapController, - config.replicationMysql, - config.replicaServerID, - config.databaseName, - context.getMetrics(), - initPosition, - false, - config.clientID, - context.getHeartbeatNotifier(), - config.scripting, - context.getFilter(), - context.getConfig().getIgnoreMissingSchema(), - config.outputConfig, - config.bufferMemoryUsage, - config.replicationReconnectionRetries, - config.binlogEventQueueSize); - } + this.replicator = new BinlogConnectorReplicator( + mysqlSchemaStore, + producer, + bootstrapController, + config.replicationMysql, + config.replicaServerID, + config.databaseName, + context.getMetrics(), + initPosition, + false, + config.clientID, + context.getHeartbeatNotifier(), + config.scripting, + context.getFilter(), + context.getConfig().getIgnoreMissingSchema(), + config.outputConfig, + config.bufferMemoryUsage, + config.replicationReconnectionRetries, + config.binlogEventQueueSize); + } context.setReplicator(replicator); this.context.start(); From d2cbc200f9e94b0c2b072fcf106cb7ba42fe312f Mon Sep 17 00:00:00 2001 From: Oleksiy Kovyrin Date: Tue, 22 Nov 2022 12:50:28 -0500 Subject: [PATCH 05/32] Add Vitess position tracking/recovery support --- .../java/com/zendesk/maxwell/Maxwell.java | 33 ++++---- .../com/zendesk/maxwell/MaxwellContext.java | 6 +- .../zendesk/maxwell/replication/Position.java | 50 ++++++++++-- .../replication/VStreamReplicator.java | 76 +++++++++++++----- .../java/com/zendesk/maxwell/row/RowMap.java | 5 ++ .../maxwell/schema/MysqlPositionStore.java | 8 +- .../maxwell/schema/VitessPositionStore.java | 77 +++++++++++++++---- src/main/resources/sql/maxwell_schema.sql | 1 + 8 files changed, 198 insertions(+), 58 deletions(-) diff --git a/src/main/java/com/zendesk/maxwell/Maxwell.java b/src/main/java/com/zendesk/maxwell/Maxwell.java index 86cc99706..e8f9a68eb 100644 --- a/src/main/java/com/zendesk/maxwell/Maxwell.java +++ b/src/main/java/com/zendesk/maxwell/Maxwell.java @@ -175,7 +175,7 @@ protected Position getInitialPosition() throws Exception { } /* fourth method: capture the current master position. */ - if ( initial == null ) { + if (initial == null && !config.vitessEnabled) { try ( Connection c = context.getReplicationConnection() ) { initial = Position.capture(c, config.gtidMode); } @@ -204,7 +204,8 @@ public String getMaxwellVersion() { static String bootString = "Maxwell v%s is booting (%s), starting at %s"; private void logBanner(AbstractProducer producer, Position initialPosition) { String producerName = producer.getClass().getSimpleName(); - LOGGER.info(String.format(bootString, getMaxwellVersion(), producerName, initialPosition.toString())); + String position = initialPosition != null ? initialPosition.toString() : "the latest position"; + LOGGER.info(String.format(bootString, getMaxwellVersion(), producerName, position)); } /** @@ -252,15 +253,7 @@ private void startInner() throws Exception { AbstractProducer producer = this.context.getProducer(); - if (config.vitessEnabled) { - this.replicator = new VStreamReplicator( - config.vitessConfig, - producer, - context.getMetrics(), - context.getFilter(), - config.bufferMemoryUsage, - config.binlogEventQueueSize); - } else { + if (!config.vitessEnabled) { try (Connection connection = context.getReplicationConnection()) { MaxwellMysqlStatus.ensureReplicationMysqlState(connection); @@ -268,11 +261,23 @@ private void startInner() throws Exception { MaxwellMysqlStatus.ensureGtidMysqlState(connection); } } + } - Position initPosition = getInitialPosition(); - logBanner(producer, initPosition); - this.context.setPosition(initPosition); + Position initPosition = getInitialPosition(); + logBanner(producer, initPosition); + this.context.setPosition(initPosition); + if (config.vitessEnabled) { + this.replicator = new VStreamReplicator( + config.vitessConfig, + producer, + context.getPositionStore(), + initPosition, + context.getMetrics(), + context.getFilter(), + config.bufferMemoryUsage, + config.binlogEventQueueSize); + } else { MysqlSchemaStore mysqlSchemaStore = new MysqlSchemaStore(this.context, initPosition); BootstrapController bootstrapController = this.context .getBootstrapController(mysqlSchemaStore.getSchemaID()); diff --git a/src/main/java/com/zendesk/maxwell/MaxwellContext.java b/src/main/java/com/zendesk/maxwell/MaxwellContext.java index 57ddc0b56..065ad4eb9 100644 --- a/src/main/java/com/zendesk/maxwell/MaxwellContext.java +++ b/src/main/java/com/zendesk/maxwell/MaxwellContext.java @@ -137,9 +137,7 @@ public MaxwellContext(MaxwellConfig config) throws SQLException, URISyntaxExcept } else if (this.config.vitessEnabled) { this.positionStore = new VitessPositionStore( this.getMaxwellConnectionPool(), - this.getServerID(), - this.config.clientID, - config.gtidMode); + this.config.clientID); } else { this.positionStore = new MysqlPositionStore( this.getMaxwellConnectionPool(), @@ -270,7 +268,7 @@ public Thread terminate(Exception error) { } if (taskManager.requestStop()) { - if (this.error == null && this.replicator != null) { + if (this.error == null && this.replicator != null && !config.vitessEnabled) { sendFinalHeartbeat(); } this.terminationThread = spawnTerminateThread(); diff --git a/src/main/java/com/zendesk/maxwell/replication/Position.java b/src/main/java/com/zendesk/maxwell/replication/Position.java index 3ed0a4c78..49ae86715 100644 --- a/src/main/java/com/zendesk/maxwell/replication/Position.java +++ b/src/main/java/com/zendesk/maxwell/replication/Position.java @@ -4,21 +4,37 @@ import java.sql.Connection; import java.sql.SQLException; +import com.zendesk.maxwell.replication.vitess.Vgtid; + public class Position implements Serializable { // LastHeartbeat is the most recent heartbeat seen prior to this position. // For a HeartbeatRow, it is the exact (new) heartbeat value for this position. private final long lastHeartbeatRead; private final BinlogPosition binlogPosition; + private final Vgtid vgtid; public Position(BinlogPosition binlogPosition, long lastHeartbeatRead) { this.binlogPosition = binlogPosition; this.lastHeartbeatRead = lastHeartbeatRead; + this.vgtid = null; + } + + // Vitess-related constructor + // FIXME: We may want to introduce a separate position class for Vitess + public Position(Vgtid vgtid) { + this.binlogPosition = null; + this.lastHeartbeatRead = 0L; + this.vgtid = vgtid; } public static Position valueOf(BinlogPosition binlogPosition, Long lastHeartbeatRead) { return new Position(binlogPosition, lastHeartbeatRead); } + public static Position valueOf(Vgtid vgtid) { + return new Position(vgtid); + } + public Position withHeartbeat(long lastHeartbeatRead) { return new Position(getBinlogPosition(), lastHeartbeatRead); } @@ -35,19 +51,29 @@ public BinlogPosition getBinlogPosition() { return binlogPosition; } + public Vgtid getVgtid() { + return vgtid; + } + public Position addGtid(String gtid, long offset, String file) { return new Position(binlogPosition.addGtid(gtid, offset, file), lastHeartbeatRead); } @Override public String toString() { - return "Position[" + binlogPosition + ", lastHeartbeat=" + lastHeartbeatRead + "]"; + if (vgtid == null) { + return "Position[" + binlogPosition + ", lastHeartbeat=" + lastHeartbeatRead + "]"; + } else { + return "Position[" + vgtid + "]"; + } } public String toCommandline() { String gtid = binlogPosition.getGtidSetStr(); if ( gtid != null ) return gtid; + else if (vgtid != null) + return vgtid.toString(); else return binlogPosition.getFile() + ":" + binlogPosition.getOffset(); } @@ -59,18 +85,32 @@ public boolean equals(Object o) { } Position other = (Position) o; - return lastHeartbeatRead == other.lastHeartbeatRead - && binlogPosition.equals(other.binlogPosition); + if (vgtid != null) { + return vgtid.equals(other.vgtid); + } else { + return lastHeartbeatRead == other.lastHeartbeatRead && binlogPosition.equals(other.binlogPosition); + } } @Override public int hashCode() { - return binlogPosition.hashCode(); + if (vgtid != null) { + return vgtid.hashCode(); + } else { + return binlogPosition.hashCode(); + } } public boolean newerThan(Position other) { if ( other == null ) return true; - return this.getBinlogPosition().newerThan(other.getBinlogPosition()); + + if (vgtid != null) { + // FIXME: Implement actual newerThan comparison for Vgtid values, for now just + // check if it is different to avoid persisting the same position over and over + return !vgtid.equals(other.vgtid); + } else { + return this.getBinlogPosition().newerThan(other.getBinlogPosition()); + } } } diff --git a/src/main/java/com/zendesk/maxwell/replication/VStreamReplicator.java b/src/main/java/com/zendesk/maxwell/replication/VStreamReplicator.java index 6fcc1b63f..20972f6e2 100644 --- a/src/main/java/com/zendesk/maxwell/replication/VStreamReplicator.java +++ b/src/main/java/com/zendesk/maxwell/replication/VStreamReplicator.java @@ -1,5 +1,6 @@ package com.zendesk.maxwell.replication; +import java.sql.SQLException; import java.util.ArrayList; import java.util.List; import java.util.concurrent.LinkedBlockingDeque; @@ -13,6 +14,7 @@ import com.codahale.metrics.Meter; import com.codahale.metrics.MetricRegistry; import com.zendesk.maxwell.MaxwellVitessConfig; +import com.zendesk.maxwell.errors.DuplicateProcessException; import com.zendesk.maxwell.filtering.Filter; import com.zendesk.maxwell.monitoring.Metrics; import com.zendesk.maxwell.producer.AbstractProducer; @@ -22,8 +24,10 @@ import com.zendesk.maxwell.replication.vitess.VitessTable; import com.zendesk.maxwell.row.RowMap; import com.zendesk.maxwell.row.RowMapBuffer; +import com.zendesk.maxwell.schema.MysqlPositionStore; import com.zendesk.maxwell.schema.Schema; import com.zendesk.maxwell.schema.SchemaStoreException; +import com.zendesk.maxwell.schema.VitessPositionStore; import com.zendesk.maxwell.util.RunLoopProcess; import binlogdata.Binlogdata.FieldEvent; @@ -31,6 +35,7 @@ import binlogdata.Binlogdata.RowEvent; import binlogdata.Binlogdata.VEvent; import binlogdata.Binlogdata.VEventType; +import binlogdata.Binlogdata.VGtid; import io.grpc.ManagedChannel; import io.grpc.ManagedChannelBuilder; @@ -54,7 +59,8 @@ public class VStreamReplicator extends RunLoopProcess implements Replicator { private final MaxwellVitessConfig vitessConfig; private final AbstractProducer producer; - private final Vgtid initialVgtid; + private final Position initPosition; + private final VitessPositionStore positionStore; private RowMapBuffer rowBuffer; private final float bufferMemoryUsage; @@ -74,12 +80,16 @@ public class VStreamReplicator extends RunLoopProcess implements Replicator { public VStreamReplicator( MaxwellVitessConfig vitessConfig, AbstractProducer producer, + MysqlPositionStore positionStore, + Position initPosition, Metrics metrics, Filter filter, Float bufferMemoryUsage, int binlogEventQueueSize) { this.vitessConfig = vitessConfig; this.producer = producer; + this.initPosition = initPosition; + this.positionStore = (VitessPositionStore) positionStore; this.queue = new LinkedBlockingDeque<>(binlogEventQueueSize); this.filter = filter; this.bufferMemoryUsage = bufferMemoryUsage; @@ -90,15 +100,6 @@ public VStreamReplicator( rowMeter = mr.meter(metrics.metricName("row", "meter")); transactionRowCount = mr.histogram(metrics.metricName("transaction", "row_count")); transactionExecutionTime = mr.histogram(metrics.metricName("transaction", "execution_time")); - - // Providing a vgtid MySQL56/19eb2657-abc2-11ea-8ffc-0242ac11000a:1-61 here will - // make VStream to - // start receiving row-changes from - // MySQL56/19eb2657-abc2-11ea-8ffc-0242ac11000a:1-62 - // TODO: Need to load the latest vgtid from a persistent store. - initialVgtid = Vgtid.of( - List.of( - new Vgtid.ShardGtid(vitessConfig.keyspace, vitessConfig.shard, "current"))); } public void startReplicator() throws Exception { @@ -118,7 +119,7 @@ public void startReplicator() throws Exception { .build(); VStreamRequest vstreamRequest = VStreamRequest.newBuilder() - .setVgtid(initialVgtid.getRawVgtid()) + .setVgtid(initialVgtid()) .setTabletType(Topodata.TabletType.MASTER) .setFlags(vStreamFlags) .build(); @@ -180,7 +181,7 @@ public RowMap getRow() throws Exception { return rowBuffer.removeFirst(); } - final VEvent event = pollEvent(); + VEvent event = pollEvent(); if (event == null) { return null; } @@ -193,6 +194,19 @@ public RowMap getRow() throws Exception { } } + private VGtid initialVgtid() { + Vgtid initialVgtid; + if (initPosition != null) { + initialVgtid = initPosition.getVgtid(); + } else { + Vgtid.ShardGtid shardGtid = new Vgtid.ShardGtid(vitessConfig.keyspace, vitessConfig.shard, "current"); + initialVgtid = Vgtid.of(List.of(shardGtid)); + } + + LOGGER.debug("Setting the initial vgtid for the stream to {}", initialVgtid); + return initialVgtid.getRawVgtid(); + } + private void processRow(RowMap row) throws Exception { producer.push(row); } @@ -203,12 +217,13 @@ private VEvent pollEvent() throws InterruptedException { private void processFieldEvent(VEvent event) { FieldEvent fieldEvent = event.getFieldEvent(); - LOGGER.info("Received field event: " + fieldEvent); + LOGGER.debug("Received field event: {}", fieldEvent); vitessSchema.processFieldEvent(fieldEvent); } - private void processVGtidEvent(VEvent event) { - LOGGER.info("Received GTID event: " + event); + private Vgtid processVGtidEvent(VEvent event) { + LOGGER.debug("Received GTID event: {}", event); + return Vgtid.of(event.getVgtid()); } /** @@ -233,6 +248,10 @@ private RowMapBuffer getTransactionRows(VEvent beginEvent) throws Exception { LOGGER.debug("Generated transaction id: {}", xid); buffer.setXid(xid); + // Since specific VStream events do not provide the VGTID, we capture it from + // VGTID events present in each transaction. + Vgtid vgtid = null; + while (true) { final VEvent event = pollEvent(); if (event == null) { @@ -241,10 +260,23 @@ private RowMapBuffer getTransactionRows(VEvent beginEvent) throws Exception { final VEventType eventType = event.getType(); + if (eventType == VEventType.VGTID) { + vgtid = processVGtidEvent(event); + continue; + } + if (eventType == VEventType.COMMIT) { LOGGER.debug("Received COMMIT event"); if (!buffer.isEmpty()) { - buffer.getLast().setTXCommit(); + // Set TX flag and the position on the last row in the transaction + RowMap lastEvent = buffer.getLast(); + if (vgtid != null) { + lastEvent.setNextPosition(new Position(vgtid)); + } else { + throw new RuntimeException("VGTID is null for transaction"); + } + lastEvent.setTXCommit(); + long timeSpent = buffer.getLast().getTimestampMillis() - beginEvent.getTimestamp(); transactionExecutionTime.update(timeSpent); transactionRowCount.update(buffer.size()); @@ -266,7 +298,7 @@ private RowMapBuffer getTransactionRows(VEvent beginEvent) throws Exception { } } - private void processServiceEvent(VEvent event) { + private void processServiceEvent(VEvent event) throws SQLException, DuplicateProcessException { final VEventType eventType = event.getType(); switch (eventType) { case FIELD: @@ -278,7 +310,15 @@ private void processServiceEvent(VEvent event) { break; case VGTID: - processVGtidEvent(event); + // Use an initial VGTID event received after connecting to vtgate as for setting + // the initial position of the stream. + if (initPosition == null) { + Position position = new Position(processVGtidEvent(event)); + LOGGER.info("Current VGTID event received, using it for initial positioning at {}", event); + positionStore.set(position); + } else { + LOGGER.warn("Ignoring a standalone VGTID event, we already have an initial position: {}", event); + } break; case ROW: diff --git a/src/main/java/com/zendesk/maxwell/row/RowMap.java b/src/main/java/com/zendesk/maxwell/row/RowMap.java index 969879491..c75eca41f 100644 --- a/src/main/java/com/zendesk/maxwell/row/RowMap.java +++ b/src/main/java/com/zendesk/maxwell/row/RowMap.java @@ -318,6 +318,11 @@ public void putOldData(String key, Object value) { } public Position getNextPosition() { return nextPosition; } + + public void setNextPosition(Position p) { + this.nextPosition = p; + } + public Position getPosition() { return position; } public Long getXid() { diff --git a/src/main/java/com/zendesk/maxwell/schema/MysqlPositionStore.java b/src/main/java/com/zendesk/maxwell/schema/MysqlPositionStore.java index b61aa271d..ae94680e9 100644 --- a/src/main/java/com/zendesk/maxwell/schema/MysqlPositionStore.java +++ b/src/main/java/com/zendesk/maxwell/schema/MysqlPositionStore.java @@ -20,10 +20,10 @@ public class MysqlPositionStore { static final Logger LOGGER = LoggerFactory.getLogger(MysqlPositionStore.class); private static final Long DEFAULT_GTID_SERVER_ID = new Long(0); - private final Long serverID; - private String clientID; + protected final Long serverID; + protected String clientID; private final boolean gtidMode; - private final ConnectionPool connectionPool; + protected final ConnectionPool connectionPool; public MysqlPositionStore(ConnectionPool pool, Long serverID, String clientID, boolean gtidMode) { this.connectionPool = pool; @@ -157,7 +157,7 @@ public Long getLastHeartbeatSent() { return lastHeartbeat; } - private Position positionFromResultSet(ResultSet rs) throws SQLException { + protected Position positionFromResultSet(ResultSet rs) throws SQLException { if ( !rs.next() ) return null; diff --git a/src/main/java/com/zendesk/maxwell/schema/VitessPositionStore.java b/src/main/java/com/zendesk/maxwell/schema/VitessPositionStore.java index 8877c1b8b..b2702f94c 100644 --- a/src/main/java/com/zendesk/maxwell/schema/VitessPositionStore.java +++ b/src/main/java/com/zendesk/maxwell/schema/VitessPositionStore.java @@ -1,20 +1,71 @@ package com.zendesk.maxwell.schema; +import java.sql.PreparedStatement; +import java.sql.ResultSet; +import java.sql.SQLException; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import com.zendesk.maxwell.errors.DuplicateProcessException; import com.zendesk.maxwell.replication.Position; +import com.zendesk.maxwell.replication.vitess.Vgtid; import com.zendesk.maxwell.util.ConnectionPool; public class VitessPositionStore extends MysqlPositionStore { - public VitessPositionStore(ConnectionPool pool, Long serverID, String clientID, boolean gtidMode) { - super(pool, serverID, clientID, gtidMode); - } - - @Override - public void set(Position p) { - // TODO: implement this for storing vtgid values - } - - @Override - public long heartbeat() throws Exception { - return System.currentTimeMillis(); - } + static final Logger LOGGER = LoggerFactory.getLogger(VitessPositionStore.class); + + public VitessPositionStore(ConnectionPool pool, String clientID) { + super(pool, 0L, clientID, true); + } + + @Override + public void set(Position p) throws SQLException, DuplicateProcessException { + if (p == null) { + LOGGER.debug("Position is null, not persisting it"); + return; + } + + Vgtid vgtid = p.getVgtid(); + if (vgtid == null) { + throw new RuntimeException("Vitess position store called with a mysql position"); + } + + String sql = "INSERT INTO `positions` " + + "SET server_id = ?, client_id = ?, vgtid = ? " + + "ON DUPLICATE KEY UPDATE client_id = ?, vgtid = ?"; + + connectionPool.withSQLRetry(1, (c) -> { + try (PreparedStatement s = c.prepareStatement(sql)) { + final String vgtidStr = vgtid.toString(); + LOGGER.debug("Writing VGTID to {}.positions: {}", c.getCatalog(), vgtidStr); + + s.setLong(1, serverID); + s.setString(2, clientID); + s.setString(3, vgtidStr); + s.setString(4, clientID); + s.setString(5, vgtidStr); + + s.execute(); + } + }); + } + + @Override + public long heartbeat() throws Exception { + // Heartbeats are not supported in Vitess. + return System.currentTimeMillis(); + } + + @Override + protected Position positionFromResultSet(ResultSet rs) throws SQLException { + if (!rs.next()) { + return null; + } + + String vgtidString = rs.getString("vgtid"); + Vgtid vgtid = Vgtid.of(vgtidString); + + return new Position(vgtid); + } } diff --git a/src/main/resources/sql/maxwell_schema.sql b/src/main/resources/sql/maxwell_schema.sql index cb89699dd..3c635bfc3 100644 --- a/src/main/resources/sql/maxwell_schema.sql +++ b/src/main/resources/sql/maxwell_schema.sql @@ -52,6 +52,7 @@ CREATE TABLE IF NOT EXISTS `positions` ( binlog_file varchar(255), binlog_position int unsigned, gtid_set varchar(4096), + vgtid text charset latin1, client_id varchar(255) charset latin1 not null default 'maxwell', heartbeat_at bigint null default null, last_heartbeat_read bigint null default null, From d1404274e819ce09622480f693dee27513b0f831 Mon Sep 17 00:00:00 2001 From: Oleksiy Kovyrin Date: Tue, 22 Nov 2022 17:09:32 -0500 Subject: [PATCH 06/32] Cleanup Position classes (move Vitess stuff into a dedicated class) --- .../BinlogConnectorReplicator.java | 3 +- .../zendesk/maxwell/replication/Position.java | 60 ++++--------------- .../replication/VStreamReplicator.java | 20 +++---- .../maxwell/replication/VitessPosition.java | 50 ++++++++++++++++ .../maxwell/schema/VitessPositionStore.java | 6 +- 5 files changed, 76 insertions(+), 63 deletions(-) create mode 100644 src/main/java/com/zendesk/maxwell/replication/VitessPosition.java diff --git a/src/main/java/com/zendesk/maxwell/replication/BinlogConnectorReplicator.java b/src/main/java/com/zendesk/maxwell/replication/BinlogConnectorReplicator.java index 9eb5a62fc..d19f35d81 100644 --- a/src/main/java/com/zendesk/maxwell/replication/BinlogConnectorReplicator.java +++ b/src/main/java/com/zendesk/maxwell/replication/BinlogConnectorReplicator.java @@ -338,7 +338,8 @@ protected void processRow(RowMap row) throws Exception { if ( row instanceof HeartbeatRowMap) { producer.push(row); if (stopAtHeartbeat != null) { - long thisHeartbeat = row.getPosition().getLastHeartbeatRead(); + Position p = (Position) row.getPosition(); + long thisHeartbeat = p.getLastHeartbeatRead(); if (thisHeartbeat >= stopAtHeartbeat) { LOGGER.info("received final heartbeat " + thisHeartbeat + "; stopping replicator"); // terminate runLoop diff --git a/src/main/java/com/zendesk/maxwell/replication/Position.java b/src/main/java/com/zendesk/maxwell/replication/Position.java index 49ae86715..9a3d5998e 100644 --- a/src/main/java/com/zendesk/maxwell/replication/Position.java +++ b/src/main/java/com/zendesk/maxwell/replication/Position.java @@ -4,37 +4,21 @@ import java.sql.Connection; import java.sql.SQLException; -import com.zendesk.maxwell.replication.vitess.Vgtid; - public class Position implements Serializable { // LastHeartbeat is the most recent heartbeat seen prior to this position. // For a HeartbeatRow, it is the exact (new) heartbeat value for this position. private final long lastHeartbeatRead; private final BinlogPosition binlogPosition; - private final Vgtid vgtid; public Position(BinlogPosition binlogPosition, long lastHeartbeatRead) { this.binlogPosition = binlogPosition; this.lastHeartbeatRead = lastHeartbeatRead; - this.vgtid = null; - } - - // Vitess-related constructor - // FIXME: We may want to introduce a separate position class for Vitess - public Position(Vgtid vgtid) { - this.binlogPosition = null; - this.lastHeartbeatRead = 0L; - this.vgtid = vgtid; } public static Position valueOf(BinlogPosition binlogPosition, Long lastHeartbeatRead) { return new Position(binlogPosition, lastHeartbeatRead); } - public static Position valueOf(Vgtid vgtid) { - return new Position(vgtid); - } - public Position withHeartbeat(long lastHeartbeatRead) { return new Position(getBinlogPosition(), lastHeartbeatRead); } @@ -51,66 +35,42 @@ public BinlogPosition getBinlogPosition() { return binlogPosition; } - public Vgtid getVgtid() { - return vgtid; - } - public Position addGtid(String gtid, long offset, String file) { return new Position(binlogPosition.addGtid(gtid, offset, file), lastHeartbeatRead); } @Override public String toString() { - if (vgtid == null) { - return "Position[" + binlogPosition + ", lastHeartbeat=" + lastHeartbeatRead + "]"; - } else { - return "Position[" + vgtid + "]"; - } + return "Position[" + binlogPosition + ", lastHeartbeat=" + lastHeartbeatRead + "]"; } public String toCommandline() { String gtid = binlogPosition.getGtidSetStr(); - if ( gtid != null ) + if (gtid != null) return gtid; - else if (vgtid != null) - return vgtid.toString(); else return binlogPosition.getFile() + ":" + binlogPosition.getOffset(); } @Override public boolean equals(Object o) { - if ( !(o instanceof Position) ) { - return false; - } - Position other = (Position) o; - - if (vgtid != null) { - return vgtid.equals(other.vgtid); - } else { + if (o instanceof Position) { + Position other = (Position) o; return lastHeartbeatRead == other.lastHeartbeatRead && binlogPosition.equals(other.binlogPosition); } + return false; } @Override public int hashCode() { - if (vgtid != null) { - return vgtid.hashCode(); - } else { - return binlogPosition.hashCode(); - } + return binlogPosition.hashCode(); } public boolean newerThan(Position other) { - if ( other == null ) - return true; - - if (vgtid != null) { - // FIXME: Implement actual newerThan comparison for Vgtid values, for now just - // check if it is different to avoid persisting the same position over and over - return !vgtid.equals(other.vgtid); - } else { - return this.getBinlogPosition().newerThan(other.getBinlogPosition()); + if (other instanceof Position) { + Position mOther = (Position) other; + return binlogPosition.newerThan(mOther.binlogPosition); } + return true; } } diff --git a/src/main/java/com/zendesk/maxwell/replication/VStreamReplicator.java b/src/main/java/com/zendesk/maxwell/replication/VStreamReplicator.java index 20972f6e2..dc6750fcb 100644 --- a/src/main/java/com/zendesk/maxwell/replication/VStreamReplicator.java +++ b/src/main/java/com/zendesk/maxwell/replication/VStreamReplicator.java @@ -59,7 +59,7 @@ public class VStreamReplicator extends RunLoopProcess implements Replicator { private final MaxwellVitessConfig vitessConfig; private final AbstractProducer producer; - private final Position initPosition; + private final VitessPosition initPosition; private final VitessPositionStore positionStore; private RowMapBuffer rowBuffer; private final float bufferMemoryUsage; @@ -88,7 +88,7 @@ public VStreamReplicator( int binlogEventQueueSize) { this.vitessConfig = vitessConfig; this.producer = producer; - this.initPosition = initPosition; + this.initPosition = (VitessPosition) initPosition; this.positionStore = (VitessPositionStore) positionStore; this.queue = new LinkedBlockingDeque<>(binlogEventQueueSize); this.filter = filter; @@ -221,9 +221,9 @@ private void processFieldEvent(VEvent event) { vitessSchema.processFieldEvent(fieldEvent); } - private Vgtid processVGtidEvent(VEvent event) { + private VitessPosition processVGtidEvent(VEvent event) { LOGGER.debug("Received GTID event: {}", event); - return Vgtid.of(event.getVgtid()); + return new VitessPosition(Vgtid.of(event.getVgtid())); } /** @@ -250,7 +250,7 @@ private RowMapBuffer getTransactionRows(VEvent beginEvent) throws Exception { // Since specific VStream events do not provide the VGTID, we capture it from // VGTID events present in each transaction. - Vgtid vgtid = null; + VitessPosition latestPosition = null; while (true) { final VEvent event = pollEvent(); @@ -261,7 +261,7 @@ private RowMapBuffer getTransactionRows(VEvent beginEvent) throws Exception { final VEventType eventType = event.getType(); if (eventType == VEventType.VGTID) { - vgtid = processVGtidEvent(event); + latestPosition = processVGtidEvent(event); continue; } @@ -270,8 +270,8 @@ private RowMapBuffer getTransactionRows(VEvent beginEvent) throws Exception { if (!buffer.isEmpty()) { // Set TX flag and the position on the last row in the transaction RowMap lastEvent = buffer.getLast(); - if (vgtid != null) { - lastEvent.setNextPosition(new Position(vgtid)); + if (latestPosition != null) { + lastEvent.setNextPosition(latestPosition); } else { throw new RuntimeException("VGTID is null for transaction"); } @@ -313,8 +313,8 @@ private void processServiceEvent(VEvent event) throws SQLException, DuplicatePro // Use an initial VGTID event received after connecting to vtgate as for setting // the initial position of the stream. if (initPosition == null) { - Position position = new Position(processVGtidEvent(event)); - LOGGER.info("Current VGTID event received, using it for initial positioning at {}", event); + VitessPosition position = processVGtidEvent(event); + LOGGER.info("Current VGTID event received, using it for initial positioning at {}", position); positionStore.set(position); } else { LOGGER.warn("Ignoring a standalone VGTID event, we already have an initial position: {}", event); diff --git a/src/main/java/com/zendesk/maxwell/replication/VitessPosition.java b/src/main/java/com/zendesk/maxwell/replication/VitessPosition.java new file mode 100644 index 000000000..4e77acd5d --- /dev/null +++ b/src/main/java/com/zendesk/maxwell/replication/VitessPosition.java @@ -0,0 +1,50 @@ +package com.zendesk.maxwell.replication; + +import com.zendesk.maxwell.replication.vitess.Vgtid; + +public class VitessPosition extends Position { + private final Vgtid vgtid; + + public VitessPosition(Vgtid vgtid) { + super(null, 0L); + this.vgtid = vgtid; + } + + public static VitessPosition valueOf(Vgtid vgtid) { + return new VitessPosition(vgtid); + } + + public Vgtid getVgtid() { + return vgtid; + } + + @Override + public String toString() { + return "Position[" + vgtid + "]"; + } + + @Override + public boolean equals(Object o) { + if (o instanceof VitessPosition) { + VitessPosition other = (VitessPosition) o; + return vgtid.equals(other.vgtid); + } + return false; + } + + @Override + public int hashCode() { + return vgtid.hashCode(); + } + + public boolean newerThan(Position other) { + if (other instanceof VitessPosition) { + // FIXME: Implement actual newerThan comparison for Vgtid values + // For now just check if it is different to avoid persisting the same position + // multiple times + VitessPosition vOther = (VitessPosition) other; + return !vgtid.equals(vOther.vgtid); + } + return true; + } +} diff --git a/src/main/java/com/zendesk/maxwell/schema/VitessPositionStore.java b/src/main/java/com/zendesk/maxwell/schema/VitessPositionStore.java index b2702f94c..406070aa4 100644 --- a/src/main/java/com/zendesk/maxwell/schema/VitessPositionStore.java +++ b/src/main/java/com/zendesk/maxwell/schema/VitessPositionStore.java @@ -9,6 +9,7 @@ import com.zendesk.maxwell.errors.DuplicateProcessException; import com.zendesk.maxwell.replication.Position; +import com.zendesk.maxwell.replication.VitessPosition; import com.zendesk.maxwell.replication.vitess.Vgtid; import com.zendesk.maxwell.util.ConnectionPool; @@ -26,7 +27,8 @@ public void set(Position p) throws SQLException, DuplicateProcessException { return; } - Vgtid vgtid = p.getVgtid(); + VitessPosition vp = (VitessPosition) p; + Vgtid vgtid = vp.getVgtid(); if (vgtid == null) { throw new RuntimeException("Vitess position store called with a mysql position"); } @@ -66,6 +68,6 @@ protected Position positionFromResultSet(ResultSet rs) throws SQLException { String vgtidString = rs.getString("vgtid"); Vgtid vgtid = Vgtid.of(vgtidString); - return new Position(vgtid); + return new VitessPosition(vgtid); } } From 7c7abefe67366df4cd7f488d4c247a347bf0d0ad Mon Sep 17 00:00:00 2001 From: Oleksiy Kovyrin Date: Tue, 22 Nov 2022 17:30:14 -0500 Subject: [PATCH 07/32] Revert some formatting changes introduced by VScode --- .../java/com/zendesk/maxwell/Maxwell.java | 56 ++++++++++--------- .../com/zendesk/maxwell/MaxwellContext.java | 26 +++------ .../zendesk/maxwell/replication/Position.java | 2 +- 3 files changed, 39 insertions(+), 45 deletions(-) diff --git a/src/main/java/com/zendesk/maxwell/Maxwell.java b/src/main/java/com/zendesk/maxwell/Maxwell.java index e8f9a68eb..4a358839c 100644 --- a/src/main/java/com/zendesk/maxwell/Maxwell.java +++ b/src/main/java/com/zendesk/maxwell/Maxwell.java @@ -175,7 +175,7 @@ protected Position getInitialPosition() throws Exception { } /* fourth method: capture the current master position. */ - if (initial == null && !config.vitessEnabled) { + if ( initial == null && !config.vitessEnabled ) { try ( Connection c = context.getReplicationConnection() ) { initial = Position.capture(c, config.gtidMode); } @@ -269,14 +269,15 @@ private void startInner() throws Exception { if (config.vitessEnabled) { this.replicator = new VStreamReplicator( - config.vitessConfig, - producer, - context.getPositionStore(), - initPosition, - context.getMetrics(), - context.getFilter(), - config.bufferMemoryUsage, - config.binlogEventQueueSize); + config.vitessConfig, + producer, + context.getPositionStore(), + initPosition, + context.getMetrics(), + context.getFilter(), + config.bufferMemoryUsage, + config.binlogEventQueueSize + ); } else { MysqlSchemaStore mysqlSchemaStore = new MysqlSchemaStore(this.context, initPosition); BootstrapController bootstrapController = this.context @@ -291,24 +292,25 @@ private void startInner() throws Exception { mysqlSchemaStore.getSchema(); // trigger schema to load / capture before we start the replicator. this.replicator = new BinlogConnectorReplicator( - mysqlSchemaStore, - producer, - bootstrapController, - config.replicationMysql, - config.replicaServerID, - config.databaseName, - context.getMetrics(), - initPosition, - false, - config.clientID, - context.getHeartbeatNotifier(), - config.scripting, - context.getFilter(), - context.getConfig().getIgnoreMissingSchema(), - config.outputConfig, - config.bufferMemoryUsage, - config.replicationReconnectionRetries, - config.binlogEventQueueSize); + mysqlSchemaStore, + producer, + bootstrapController, + config.replicationMysql, + config.replicaServerID, + config.databaseName, + context.getMetrics(), + initPosition, + false, + config.clientID, + context.getHeartbeatNotifier(), + config.scripting, + context.getFilter(), + context.getConfig().getIgnoreMissingSchema(), + config.outputConfig, + config.bufferMemoryUsage, + config.replicationReconnectionRetries, + config.binlogEventQueueSize + ); } context.setReplicator(replicator); diff --git a/src/main/java/com/zendesk/maxwell/MaxwellContext.java b/src/main/java/com/zendesk/maxwell/MaxwellContext.java index 065ad4eb9..29668ef37 100644 --- a/src/main/java/com/zendesk/maxwell/MaxwellContext.java +++ b/src/main/java/com/zendesk/maxwell/MaxwellContext.java @@ -128,22 +128,13 @@ public MaxwellContext(MaxwellConfig config) throws SQLException, URISyntaxExcept if ( this.config.initPosition != null ) this.initialPosition = this.config.initPosition; - if ( this.config.replayMode ) { - this.positionStore = new ReadOnlyMysqlPositionStore( - this.getMaxwellConnectionPool(), - this.getServerID(), - this.config.clientID, - config.gtidMode); - } else if (this.config.vitessEnabled) { - this.positionStore = new VitessPositionStore( - this.getMaxwellConnectionPool(), - this.config.clientID); + ConnectionPool cp = getMaxwellConnectionPool(); + if (this.config.replayMode) { + this.positionStore = new ReadOnlyMysqlPositionStore(cp, getServerID(), config.clientID, config.gtidMode); + } else if (config.vitessEnabled) { + this.positionStore = new VitessPositionStore(cp, config.clientID); } else { - this.positionStore = new MysqlPositionStore( - this.getMaxwellConnectionPool(), - this.getServerID(), - this.config.clientID, - config.gtidMode); + this.positionStore = new MysqlPositionStore(cp, getServerID(), config.clientID, config.gtidMode); } this.heartbeatNotifier = new HeartbeatNotifier(); @@ -478,8 +469,9 @@ public Long getServerID() throws SQLException { return this.serverID; try ( Connection c = getReplicationConnection(); - Statement s = c.createStatement(); - ResultSet rs = s.executeQuery("SELECT @@server_id as server_id")) { + Statement s = c.createStatement(); + ResultSet rs = s.executeQuery("SELECT @@server_id as server_id") ) + { if ( !rs.next() ) { throw new RuntimeException("Could not retrieve server_id!"); } diff --git a/src/main/java/com/zendesk/maxwell/replication/Position.java b/src/main/java/com/zendesk/maxwell/replication/Position.java index 9a3d5998e..f5c3f0af5 100644 --- a/src/main/java/com/zendesk/maxwell/replication/Position.java +++ b/src/main/java/com/zendesk/maxwell/replication/Position.java @@ -46,7 +46,7 @@ public String toString() { public String toCommandline() { String gtid = binlogPosition.getGtidSetStr(); - if (gtid != null) + if ( gtid != null ) return gtid; else return binlogPosition.getFile() + ":" + binlogPosition.getOffset(); From 60bf28e4b14611eda7d2f91b2236ea16e97a47d6 Mon Sep 17 00:00:00 2001 From: Oleksiy Kovyrin Date: Tue, 22 Nov 2022 17:31:34 -0500 Subject: [PATCH 08/32] Another VScode change --- src/main/java/com/zendesk/maxwell/Maxwell.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/main/java/com/zendesk/maxwell/Maxwell.java b/src/main/java/com/zendesk/maxwell/Maxwell.java index 4a358839c..d5ef9d814 100644 --- a/src/main/java/com/zendesk/maxwell/Maxwell.java +++ b/src/main/java/com/zendesk/maxwell/Maxwell.java @@ -246,7 +246,7 @@ private void startInner() throws Exception { SchemaStoreSchema.ensureMaxwellSchema(rawConnection, this.config.databaseName); - try (Connection schemaConnection = this.context.getMaxwellConnection()) { + try ( Connection schemaConnection = this.context.getMaxwellConnection() ) { SchemaStoreSchema.upgradeSchemaStoreSchema(schemaConnection); } } From 4dd9da7bacb402c70ac29a4b0ccca370d37a40e1 Mon Sep 17 00:00:00 2001 From: Oleksiy Kovyrin Date: Tue, 22 Nov 2022 17:33:12 -0500 Subject: [PATCH 09/32] Another VScode change --- src/main/java/com/zendesk/maxwell/MaxwellContext.java | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/src/main/java/com/zendesk/maxwell/MaxwellContext.java b/src/main/java/com/zendesk/maxwell/MaxwellContext.java index 29668ef37..8a493470b 100644 --- a/src/main/java/com/zendesk/maxwell/MaxwellContext.java +++ b/src/main/java/com/zendesk/maxwell/MaxwellContext.java @@ -469,9 +469,8 @@ public Long getServerID() throws SQLException { return this.serverID; try ( Connection c = getReplicationConnection(); - Statement s = c.createStatement(); - ResultSet rs = s.executeQuery("SELECT @@server_id as server_id") ) - { + Statement s = c.createStatement(); + ResultSet rs = s.executeQuery("SELECT @@server_id as server_id") ) { if ( !rs.next() ) { throw new RuntimeException("Could not retrieve server_id!"); } From a3ea1aa8e3f25a69f3b063a9a2e371f7aac84bb7 Mon Sep 17 00:00:00 2001 From: Oleksiy Kovyrin Date: Wed, 23 Nov 2022 09:54:18 -0500 Subject: [PATCH 10/32] Add a schema migration and change the vgtid column in positions --- .../java/com/zendesk/maxwell/schema/SchemaStoreSchema.java | 4 ++++ .../com/zendesk/maxwell/schema/VitessPositionStore.java | 6 +++--- 2 files changed, 7 insertions(+), 3 deletions(-) diff --git a/src/main/java/com/zendesk/maxwell/schema/SchemaStoreSchema.java b/src/main/java/com/zendesk/maxwell/schema/SchemaStoreSchema.java index ea602adc3..baed9fd41 100644 --- a/src/main/java/com/zendesk/maxwell/schema/SchemaStoreSchema.java +++ b/src/main/java/com/zendesk/maxwell/schema/SchemaStoreSchema.java @@ -109,6 +109,10 @@ private static void performAlter(Connection c, String sql) throws SQLException { public static void upgradeSchemaStoreSchema(Connection c) throws SQLException, IOException { ArrayList maxwellTables = getMaxwellTables(c); + if ( !getTableColumns("positions", c).containsKey("vitess_gtid") ) { + performAlter(c, "alter table `positions` add column vitess_gtid text charset latin1"); + } + if ( !getTableColumns("schemas", c).containsKey("deleted") ) { performAlter(c, "alter table `schemas` add column deleted tinyint(1) not null default 0"); } diff --git a/src/main/java/com/zendesk/maxwell/schema/VitessPositionStore.java b/src/main/java/com/zendesk/maxwell/schema/VitessPositionStore.java index 406070aa4..5882a50e4 100644 --- a/src/main/java/com/zendesk/maxwell/schema/VitessPositionStore.java +++ b/src/main/java/com/zendesk/maxwell/schema/VitessPositionStore.java @@ -34,8 +34,8 @@ public void set(Position p) throws SQLException, DuplicateProcessException { } String sql = "INSERT INTO `positions` " - + "SET server_id = ?, client_id = ?, vgtid = ? " - + "ON DUPLICATE KEY UPDATE client_id = ?, vgtid = ?"; + + "SET server_id = ?, client_id = ?, vitess_gtid = ? " + + "ON DUPLICATE KEY UPDATE client_id = ?, vitess_gtid = ?"; connectionPool.withSQLRetry(1, (c) -> { try (PreparedStatement s = c.prepareStatement(sql)) { @@ -65,7 +65,7 @@ protected Position positionFromResultSet(ResultSet rs) throws SQLException { return null; } - String vgtidString = rs.getString("vgtid"); + String vgtidString = rs.getString("vitess_gtid"); Vgtid vgtid = Vgtid.of(vgtidString); return new VitessPosition(vgtid); From eee6f0174471a6b3b0b52818fe7ca3ddf0832648 Mon Sep 17 00:00:00 2001 From: Oleksiy Kovyrin Date: Thu, 24 Nov 2022 09:32:10 -0500 Subject: [PATCH 11/32] Stop VStream replicator if the observer dies for whatever reason. --- .../zendesk/maxwell/replication/VStreamObserver.java | 8 +++++++- .../maxwell/replication/VStreamReplicator.java | 11 +++++++++++ 2 files changed, 18 insertions(+), 1 deletion(-) diff --git a/src/main/java/com/zendesk/maxwell/replication/VStreamObserver.java b/src/main/java/com/zendesk/maxwell/replication/VStreamObserver.java index 97d16c06a..317a03532 100644 --- a/src/main/java/com/zendesk/maxwell/replication/VStreamObserver.java +++ b/src/main/java/com/zendesk/maxwell/replication/VStreamObserver.java @@ -21,6 +21,7 @@ public class VStreamObserver implements StreamObserver { private static final Logger LOGGER = LoggerFactory.getLogger(VStreamObserver.class); private final AtomicBoolean mustStop = new AtomicBoolean(false); private final LinkedBlockingDeque queue; + private Exception lastException = null; public VStreamObserver(LinkedBlockingDeque queue) { this.queue = queue; @@ -44,7 +45,8 @@ public void onNext(Vtgate.VStreamResponse response) { @Override public void onError(Throwable t) { - LOGGER.error("VStream streaming onError. Status: " + Status.fromThrowable(t), t); + this.lastException = Status.fromThrowable(t).asException(); + LOGGER.error("VStream streaming onError. Status: {}", lastException); stop(); } @@ -54,6 +56,10 @@ public void onCompleted() { stop(); } + public Exception getLastException() { + return lastException; + } + // Pushes an event to the queue for VStreamReplicator to process. private void enqueueEvent(VEvent event) { while (mustStop.get() != true) { diff --git a/src/main/java/com/zendesk/maxwell/replication/VStreamReplicator.java b/src/main/java/com/zendesk/maxwell/replication/VStreamReplicator.java index dc6750fcb..babd56e5e 100644 --- a/src/main/java/com/zendesk/maxwell/replication/VStreamReplicator.java +++ b/src/main/java/com/zendesk/maxwell/replication/VStreamReplicator.java @@ -158,6 +158,7 @@ public void work() throws Exception { } if (row == null) { + checkIfVstreamIsAlive(); return; } @@ -170,6 +171,16 @@ public void work() throws Exception { processRow(row); } + private void checkIfVstreamIsAlive() { + if (!replicatorStarted) return; + + Exception lastException = responseObserver.getLastException(); + if (lastException != null) { + LOGGER.error("VStream is dead, stopping..."); + throw new RuntimeException(lastException); + } + } + public RowMap getRow() throws Exception { if (!replicatorStarted) { LOGGER.warn("replicator was not started, calling startReplicator()..."); From 236092eb25f765779444e0bebb91cf8e577368cb Mon Sep 17 00:00:00 2001 From: Oleksiy Kovyrin Date: Thu, 24 Nov 2022 09:45:40 -0500 Subject: [PATCH 12/32] When running against Vitess, we cannot rely on read_only variable --- src/main/java/com/zendesk/maxwell/Maxwell.java | 2 +- .../java/com/zendesk/maxwell/MaxwellMysqlStatus.java | 9 +++++++-- 2 files changed, 8 insertions(+), 3 deletions(-) diff --git a/src/main/java/com/zendesk/maxwell/Maxwell.java b/src/main/java/com/zendesk/maxwell/Maxwell.java index d5ef9d814..5d0c904c7 100644 --- a/src/main/java/com/zendesk/maxwell/Maxwell.java +++ b/src/main/java/com/zendesk/maxwell/Maxwell.java @@ -242,7 +242,7 @@ public void start() throws Exception { private void startInner() throws Exception { try (Connection rawConnection = this.context.getRawMaxwellConnection()) { - MaxwellMysqlStatus.ensureMaxwellMysqlState(rawConnection); + MaxwellMysqlStatus.ensureMaxwellMysqlState(rawConnection, config.vitessEnabled); SchemaStoreSchema.ensureMaxwellSchema(rawConnection, this.config.databaseName); diff --git a/src/main/java/com/zendesk/maxwell/MaxwellMysqlStatus.java b/src/main/java/com/zendesk/maxwell/MaxwellMysqlStatus.java index 486cf23c8..ab04e6cd8 100644 --- a/src/main/java/com/zendesk/maxwell/MaxwellMysqlStatus.java +++ b/src/main/java/com/zendesk/maxwell/MaxwellMysqlStatus.java @@ -109,13 +109,18 @@ public static void ensureReplicationMysqlState(Connection c) throws SQLException /** * Verify that the maxwell database is in the expected state * @param c a JDBC connection + * @param vitessEnabled whether or not we are running in a vitess environment * @throws SQLException if we have database issues * @throws MaxwellCompatibilityError if we're not in the expected state */ - public static void ensureMaxwellMysqlState(Connection c) throws SQLException, MaxwellCompatibilityError { + public static void ensureMaxwellMysqlState(Connection c, boolean vitessEnabled) throws SQLException, MaxwellCompatibilityError { MaxwellMysqlStatus m = new MaxwellMysqlStatus(c); - m.ensureVariableState("read_only", "OFF"); + // Vitess reports read_only=ON while running in a vttestserver and sometimes even + // in production. We need to ignore this setting when running against Vitess. + if (!vitessEnabled) { + m.ensureVariableState("read_only", "OFF"); + } } /** From 6e7a4d9603dd6ddf5fda470d4cb7a2f59d7e37fd Mon Sep 17 00:00:00 2001 From: Oleksiy Kovyrin Date: Thu, 24 Nov 2022 10:22:13 -0500 Subject: [PATCH 13/32] Unsupported event is not a warning --- .../com/zendesk/maxwell/replication/VStreamReplicator.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/main/java/com/zendesk/maxwell/replication/VStreamReplicator.java b/src/main/java/com/zendesk/maxwell/replication/VStreamReplicator.java index babd56e5e..a0b30e1c9 100644 --- a/src/main/java/com/zendesk/maxwell/replication/VStreamReplicator.java +++ b/src/main/java/com/zendesk/maxwell/replication/VStreamReplicator.java @@ -339,10 +339,9 @@ private void processServiceEvent(VEvent event) throws SQLException, DuplicatePro break; default: - LOGGER.warn("Unexpected service event: {}", event); + LOGGER.debug("Unsupported service event: {}", event); break; } - } private List rowEventToMaps(VEvent event, long xid) { From 918efaad2962d30bd2d01d72e55671bfba679a30 Mon Sep 17 00:00:00 2001 From: Oleksiy Kovyrin Date: Thu, 24 Nov 2022 10:22:37 -0500 Subject: [PATCH 14/32] Add Vitess test scripts, docs, etc --- vitess/01-start-vitess.sh | 20 ++++++++++++ vitess/02-start-maxwell.sh | 6 ++++ vitess/README.md | 63 ++++++++++++++++++++++++++++++++++++++ vitess/config.properties | 23 ++++++++++++++ vitess/docker-compose.yml | 19 ++++++++++++ 5 files changed, 131 insertions(+) create mode 100755 vitess/01-start-vitess.sh create mode 100755 vitess/02-start-maxwell.sh create mode 100644 vitess/README.md create mode 100644 vitess/config.properties create mode 100644 vitess/docker-compose.yml diff --git a/vitess/01-start-vitess.sh b/vitess/01-start-vitess.sh new file mode 100755 index 000000000..dd25c79fa --- /dev/null +++ b/vitess/01-start-vitess.sh @@ -0,0 +1,20 @@ +#!/usr/bin/env bash + +set -x + +DOCKER_BIN=$(which docker) +if [[ "$DOCKER_BIN" != "" ]]; then + echo "Docker is installed" + COMPOSE_COMMAND="docker compose" +else + echo "Docker is not installed, trying podman..." + PODMAN_BIN=$(which podman) + if [ -z "$PODMAN_BIN" ]; then + echo "Podman is not installed either, exiting..." + exit 1 + fi + COMPOSE_COMMAND="podman-compose" +fi + +# Start the Vitess cluster +exec $COMPOSE_COMMAND -f vitess/docker-compose.yml up diff --git a/vitess/02-start-maxwell.sh b/vitess/02-start-maxwell.sh new file mode 100755 index 000000000..51f0f9383 --- /dev/null +++ b/vitess/02-start-maxwell.sh @@ -0,0 +1,6 @@ +#!/usr/bin/env bash + +set -ex + +mvn clean package -DskipTests=true -Dmaven.javadoc.skip=true +bin/maxwell --kafka_version 2.7.0 --config vitess/config.properties diff --git a/vitess/README.md b/vitess/README.md new file mode 100644 index 000000000..84b998426 --- /dev/null +++ b/vitess/README.md @@ -0,0 +1,63 @@ +# Vitess Support for Maxwell + +This directory contains a set of scripts and configuration files to help develop +and test Vitess support in Maxwell. + +## Starting Vitess Test Server + +The easiest way to develop and test Vitess clients is by running the `vttestserver` +Docker image. It contains all of the components of Vitess + a MySQL instance configured +in a way, that mimics a real Vitess cluster close enough for most development purposes. + +See https://vitess.io/docs/13.0/get-started/vttestserver-docker-image/ for more details. + +For developing Maxwell support for Vitess, you can run run vttestserver using the +provided script: + +``` +$ ./vitess/01-start-vitess.sh +``` + +The script will use Docker or Podman to start vttestserver and expose its gRPC and MySQL ports +so that Maxwell can connect to those. + +## Running Maxwell against Vitess + +To start Maxwell against the provided vttestserver cluster, you can use the provided script: + +``` +$ ./vitess/02-start-maxwell.sh +``` +The script will build Maxwell and then start it with the provided properties file +(`vitess/config.properties`). The properties file configures the following: + +1. Maxwell's schema is stored in a dedicated Vitess keyspace called `maxwell`. +2. Maxwell follows all changes in a Vitess keyspace called `app_shard`. + +## Connecting to Vitess with a MySQL client + +To play with the provided vttestserver cluster, you can connect to it using any MySQL client. +The test server does not require any credentials. + +Here is an example command: + +``` +mysql -h 127.0.0.1 -P 33577 +``` + +The MySQL interface in vtgate will expose multiple keyspaces, which you can read and write +into as usual. + +## Connecting to VStream API using a gRPC client + +If you would like to experiment with the gRPC APIs exposed by Vitess, you can use any gRPC +client and connect to `localhost:33575`. + +Here is an example [gRPCurl](https://github.com/fullstorydev/grpcurl) command for following all +changes made to the `app_shard` keyspace: + +``` +grpcurl -plaintext -d '{"vgtid":{"shard_gtids":[{"keyspace":"app_shard", "gtid":"current"}]}}' localhost:33575 vtgateservice.Vitess.VStream +``` + +This will run the VStream API and send all events to console as a JSON stream. diff --git a/vitess/config.properties b/vitess/config.properties new file mode 100644 index 000000000..57052e473 --- /dev/null +++ b/vitess/config.properties @@ -0,0 +1,23 @@ +# +# This is a sample configuration file for running Maxwell against a Vitess cluster +# + +# Produce debug-level logging +log_level=debug + +# Send all events to console +producer=stdout + +# Enable vitess support +vitess=true +vitess_keyspace=app_shard +vitess_shard= +vitess_host=127.0.0.1 +vitess_port=33575 + +# MySQL login info for storing maxwell metadata (a separate vitess keyspace, no sharding) +host=127.0.0.1 +port=33577 +user=root +password= +schema_database=maxwell diff --git a/vitess/docker-compose.yml b/vitess/docker-compose.yml new file mode 100644 index 000000000..7c14ed4d3 --- /dev/null +++ b/vitess/docker-compose.yml @@ -0,0 +1,19 @@ +version: "3.9" +services: + vttestserver: + image: vitess/vttestserver:mysql57 + ports: + - "33575:33575" # gRPC (VTGate, etc) + - "33577:33577" # MySQL + environment: + - PORT=33574 + - PLANNER_VERSION=gen4fallback + - KEYSPACES=maxwell,app_master,app_shard,sharded_tests + - NUM_SHARDS=1,1,1,2 + - MYSQL_MAX_CONNECTIONS=70000 + - MYSQL_BIND_HOST=0.0.0.0 + healthcheck: + test: ["CMD", "mysqladmin", "ping", "-h127.0.0.1", "-P33577"] + interval: 5s + timeout: 2s + retries: 5 From 3b282ce6600f919f5286ea0b2ba07ce5209cbf7f Mon Sep 17 00:00:00 2001 From: Oleksiy Kovyrin Date: Thu, 24 Nov 2022 11:05:12 -0500 Subject: [PATCH 15/32] Intern Vitess keyspace/table/column/type names + enum values to reduce memory footprint of the in-memory schema cache --- .../replication/vitess/ColumnMetaData.java | 2 +- .../vitess/ReplicationMessageColumn.java | 2 +- .../maxwell/replication/vitess/Vgtid.java | 23 +++++++++++-------- .../replication/vitess/VitessColumn.java | 2 +- .../replication/vitess/VitessSchema.java | 2 +- .../replication/vitess/VitessTable.java | 4 ++-- .../replication/vitess/VitessType.java | 11 ++++++--- 7 files changed, 27 insertions(+), 19 deletions(-) diff --git a/src/main/java/com/zendesk/maxwell/replication/vitess/ColumnMetaData.java b/src/main/java/com/zendesk/maxwell/replication/vitess/ColumnMetaData.java index 189370d31..2ed678771 100644 --- a/src/main/java/com/zendesk/maxwell/replication/vitess/ColumnMetaData.java +++ b/src/main/java/com/zendesk/maxwell/replication/vitess/ColumnMetaData.java @@ -17,7 +17,7 @@ public class ColumnMetaData { private final KeyMetaData keyMetaData; public ColumnMetaData(String columnName, VitessType vitessType, boolean optional, KeyMetaData keyMetaData) { - this.columnName = columnName; + this.columnName = columnName.intern(); this.vitessType = vitessType; this.keyMetaData = keyMetaData; this.optional = optional; diff --git a/src/main/java/com/zendesk/maxwell/replication/vitess/ReplicationMessageColumn.java b/src/main/java/com/zendesk/maxwell/replication/vitess/ReplicationMessageColumn.java index cea365303..c24f09f68 100644 --- a/src/main/java/com/zendesk/maxwell/replication/vitess/ReplicationMessageColumn.java +++ b/src/main/java/com/zendesk/maxwell/replication/vitess/ReplicationMessageColumn.java @@ -15,7 +15,7 @@ public class ReplicationMessageColumn { private final byte[] rawValue; public ReplicationMessageColumn(String columnName, VitessType type, byte[] rawValue) { - this.columnName = columnName; + this.columnName = columnName.intern(); this.type = type; this.rawValue = rawValue; } diff --git a/src/main/java/com/zendesk/maxwell/replication/vitess/Vgtid.java b/src/main/java/com/zendesk/maxwell/replication/vitess/Vgtid.java index f59435c5b..1bd4c81d0 100644 --- a/src/main/java/com/zendesk/maxwell/replication/vitess/Vgtid.java +++ b/src/main/java/com/zendesk/maxwell/replication/vitess/Vgtid.java @@ -43,19 +43,19 @@ private Vgtid(List shardGtids) { Binlogdata.VGtid.Builder builder = Binlogdata.VGtid.newBuilder(); for (ShardGtid shardGtid : shardGtids) { builder.addShardGtids( - Binlogdata.ShardGtid.newBuilder() - .setKeyspace(shardGtid.getKeyspace()) - .setShard(shardGtid.getShard()) - .setGtid(shardGtid.getGtid()) - .build()); + Binlogdata.ShardGtid.newBuilder() + .setKeyspace(shardGtid.getKeyspace()) + .setShard(shardGtid.getShard()) + .setGtid(shardGtid.getGtid()) + .build() + ); } this.rawVgtid = builder.build(); } public static Vgtid of(String shardGtidsInJson) { try { - List shardGtids = MAPPER.readValue(shardGtidsInJson, new TypeReference>() { - }); + List shardGtids = MAPPER.readValue(shardGtidsInJson, new TypeReference>() { }); return of(shardGtids); } catch (JsonProcessingException e) { throw new IllegalStateException(e); @@ -116,9 +116,12 @@ public static class ShardGtid { private final String gtid; @JsonCreator - public ShardGtid(@JsonProperty(KEYSPACE_KEY) String keyspace, @JsonProperty(SHARD_KEY) String shard, - @JsonProperty(GTID_KEY) String gtid) { - this.keyspace = keyspace; + public ShardGtid( + @JsonProperty(KEYSPACE_KEY) String keyspace, + @JsonProperty(SHARD_KEY) String shard, + @JsonProperty(GTID_KEY) String gtid + ) { + this.keyspace = keyspace.intern(); this.shard = shard; this.gtid = gtid; } diff --git a/src/main/java/com/zendesk/maxwell/replication/vitess/VitessColumn.java b/src/main/java/com/zendesk/maxwell/replication/vitess/VitessColumn.java index 9d03e6a9b..fbfdadcf9 100644 --- a/src/main/java/com/zendesk/maxwell/replication/vitess/VitessColumn.java +++ b/src/main/java/com/zendesk/maxwell/replication/vitess/VitessColumn.java @@ -5,7 +5,7 @@ public class VitessColumn { private final VitessType type; public VitessColumn(String name, VitessType type) { - this.name = name; + this.name = name.intern(); this.type = type; } diff --git a/src/main/java/com/zendesk/maxwell/replication/vitess/VitessSchema.java b/src/main/java/com/zendesk/maxwell/replication/vitess/VitessSchema.java index 219a7a9f3..a09cabebc 100644 --- a/src/main/java/com/zendesk/maxwell/replication/vitess/VitessSchema.java +++ b/src/main/java/com/zendesk/maxwell/replication/vitess/VitessSchema.java @@ -25,7 +25,7 @@ public class VitessSchema { private final Map tables = new HashMap<>(); public void addTable(VitessTable table) { - String tableName = table.getQualifiedTableName(); + String tableName = table.getQualifiedTableName().intern(); if (tables.containsKey(tableName)) { LOGGER.info("Schema change detected for: {}", table); } else { diff --git a/src/main/java/com/zendesk/maxwell/replication/vitess/VitessTable.java b/src/main/java/com/zendesk/maxwell/replication/vitess/VitessTable.java index abb3db809..08a0b381e 100644 --- a/src/main/java/com/zendesk/maxwell/replication/vitess/VitessTable.java +++ b/src/main/java/com/zendesk/maxwell/replication/vitess/VitessTable.java @@ -15,8 +15,8 @@ public class VitessTable { private final List pkColumns; public VitessTable(String schemaName, String tableName, List columns, List pkColumns) { - this.schemaName = schemaName; - this.tableName = tableName; + this.schemaName = schemaName.intern(); + this.tableName = tableName.intern(); this.columns = columns; this.pkColumns = pkColumns; } diff --git a/src/main/java/com/zendesk/maxwell/replication/vitess/VitessType.java b/src/main/java/com/zendesk/maxwell/replication/vitess/VitessType.java index 289241352..cbc30d53d 100644 --- a/src/main/java/com/zendesk/maxwell/replication/vitess/VitessType.java +++ b/src/main/java/com/zendesk/maxwell/replication/vitess/VitessType.java @@ -10,12 +10,12 @@ import java.util.Collections; import java.util.List; import java.util.Objects; +import com.google.common.collect.ImmutableList; import io.vitess.proto.Query.Field; /** The Vitess table column type */ public class VitessType { - // name of the column type private final String name; // enum of column jdbc type @@ -28,9 +28,14 @@ public VitessType(String name, int jdbcId) { } public VitessType(String name, int jdbcId, List enumValues) { - this.name = name; + this.name = name.intern(); this.jdbcId = jdbcId; - this.enumValues = Collections.unmodifiableList(enumValues); + + ImmutableList.Builder builder = ImmutableList.builderWithExpectedSize(enumValues.size()); + for (String enumValue : enumValues) { + builder.add(enumValue.intern()); + } + this.enumValues = builder.build(); } public String getName() { From b712ae0698b0584ae74d4c14ee7f50b7872afb15 Mon Sep 17 00:00:00 2001 From: Oleksiy Kovyrin Date: Thu, 24 Nov 2022 12:13:34 -0500 Subject: [PATCH 16/32] Add a cleanup script for Vitess stuff --- vitess/01-start-vitess.sh | 5 ++++- vitess/03-cleanup-vitess.sh | 20 ++++++++++++++++++++ vitess/README.md | 13 +++++++++++-- 3 files changed, 35 insertions(+), 3 deletions(-) create mode 100755 vitess/03-cleanup-vitess.sh diff --git a/vitess/01-start-vitess.sh b/vitess/01-start-vitess.sh index dd25c79fa..2e156ee26 100755 --- a/vitess/01-start-vitess.sh +++ b/vitess/01-start-vitess.sh @@ -16,5 +16,8 @@ else COMPOSE_COMMAND="podman-compose" fi +# Stop and remove any existing containers +$COMPOSE_COMMAND -f vitess/docker-compose.yml down + # Start the Vitess cluster -exec $COMPOSE_COMMAND -f vitess/docker-compose.yml up +$COMPOSE_COMMAND -f vitess/docker-compose.yml up diff --git a/vitess/03-cleanup-vitess.sh b/vitess/03-cleanup-vitess.sh new file mode 100755 index 000000000..98ee5c7cc --- /dev/null +++ b/vitess/03-cleanup-vitess.sh @@ -0,0 +1,20 @@ +#!/usr/bin/env bash + +set -x + +DOCKER_BIN=$(which docker) +if [[ "$DOCKER_BIN" != "" ]]; then + echo "Docker is installed" + COMPOSE_COMMAND="docker compose" +else + echo "Docker is not installed, trying podman..." + PODMAN_BIN=$(which podman) + if [ -z "$PODMAN_BIN" ]; then + echo "Podman is not installed either, exiting..." + exit 1 + fi + COMPOSE_COMMAND="podman-compose" +fi + +# Stop and remove any existing containers +$COMPOSE_COMMAND -f vitess/docker-compose.yml down diff --git a/vitess/README.md b/vitess/README.md index 84b998426..96f429c85 100644 --- a/vitess/README.md +++ b/vitess/README.md @@ -15,7 +15,7 @@ For developing Maxwell support for Vitess, you can run run vttestserver using th provided script: ``` -$ ./vitess/01-start-vitess.sh +$ vitess/01-start-vitess.sh ``` The script will use Docker or Podman to start vttestserver and expose its gRPC and MySQL ports @@ -26,7 +26,7 @@ so that Maxwell can connect to those. To start Maxwell against the provided vttestserver cluster, you can use the provided script: ``` -$ ./vitess/02-start-maxwell.sh +$ vitess/02-start-maxwell.sh ``` The script will build Maxwell and then start it with the provided properties file (`vitess/config.properties`). The properties file configures the following: @@ -61,3 +61,12 @@ grpcurl -plaintext -d '{"vgtid":{"shard_gtids":[{"keyspace":"app_shard", "gtid": ``` This will run the VStream API and send all events to console as a JSON stream. + +## Cleaning up after testing + +When you're done working with the vttestserver cluster, you may want to run the script provided to +clean up any remaining Docker containers: + +``` +$ vitess/03-cleanup-vitess.sh +``` From 3265121440a77dd90bd88f9ef3222db135b4cd3c Mon Sep 17 00:00:00 2001 From: Oleksiy Kovyrin Date: Thu, 24 Nov 2022 12:13:44 -0500 Subject: [PATCH 17/32] Formatting cleanup --- .../replication/VStreamReplicator.java | 29 ++++++++++--------- 1 file changed, 15 insertions(+), 14 deletions(-) diff --git a/src/main/java/com/zendesk/maxwell/replication/VStreamReplicator.java b/src/main/java/com/zendesk/maxwell/replication/VStreamReplicator.java index a0b30e1c9..3e040a2a1 100644 --- a/src/main/java/com/zendesk/maxwell/replication/VStreamReplicator.java +++ b/src/main/java/com/zendesk/maxwell/replication/VStreamReplicator.java @@ -360,12 +360,15 @@ private List rowEventToMaps(VEvent event, long xid) { List columns = table.resolveColumns(rowChange); RowMap rowMap = new RowMap( - changeType, - table.getSchemaName(), - table.getTableName(), - timestampMillis, - table.getPkColumns(), - null, null, null); + changeType, + table.getSchemaName(), + table.getTableName(), + timestampMillis, + table.getPkColumns(), + null, + null, + null + ); rowMap.setXid(xid); @@ -403,26 +406,24 @@ private VitessTable resolveTable(String qualifiedTableName) { @Override public Long getLastHeartbeatRead() { - // TODO Auto-generated method stub - return null; + throw new RuntimeException("Heartbeat support is not available in Vitess replicator"); + } + + @Override + public void stopAtHeartbeat(long heartbeat) { + throw new RuntimeException("Heartbeat support is not available in Vitess replicator"); } @Override public Schema getSchema() throws SchemaStoreException { - // TODO Auto-generated method stub return null; } @Override public Long getSchemaId() throws SchemaStoreException { - // TODO Auto-generated method stub return null; } - @Override - public void stopAtHeartbeat(long heartbeat) { - // TODO Auto-generated method stub - } private static ManagedChannel newChannel(String vtgateHost, int vtgatePort, int maxInboundMessageSize) { return ManagedChannelBuilder From 8841ba4735490060091486a95b9b31482f1cfe7d Mon Sep 17 00:00:00 2001 From: Oleksiy Kovyrin Date: Fri, 25 Nov 2022 13:34:17 -0500 Subject: [PATCH 18/32] Add TLS support for VTgate connections --- .../zendesk/maxwell/MaxwellVitessConfig.java | 7 +++ .../replication/VStreamReplicator.java | 63 ++++++++++++++----- .../zendesk/maxwell/util/AbstractConfig.java | 10 ++- vitess/config.properties | 11 +++- 4 files changed, 70 insertions(+), 21 deletions(-) diff --git a/src/main/java/com/zendesk/maxwell/MaxwellVitessConfig.java b/src/main/java/com/zendesk/maxwell/MaxwellVitessConfig.java index 655aa4899..564937329 100644 --- a/src/main/java/com/zendesk/maxwell/MaxwellVitessConfig.java +++ b/src/main/java/com/zendesk/maxwell/MaxwellVitessConfig.java @@ -10,9 +10,16 @@ public class MaxwellVitessConfig { public String keyspace; public String shard; + public boolean usePlaintext; + public String tlsCA; + public String tlsServerName; + public MaxwellVitessConfig() { this.vtgateHost = "localhost"; this.vtgatePort = 15991; + this.usePlaintext = true; + this.tlsCA = null; + this.tlsServerName = null; this.user = null; this.password = null; diff --git a/src/main/java/com/zendesk/maxwell/replication/VStreamReplicator.java b/src/main/java/com/zendesk/maxwell/replication/VStreamReplicator.java index 3e040a2a1..2a22ace87 100644 --- a/src/main/java/com/zendesk/maxwell/replication/VStreamReplicator.java +++ b/src/main/java/com/zendesk/maxwell/replication/VStreamReplicator.java @@ -1,5 +1,7 @@ package com.zendesk.maxwell.replication; +import java.io.File; +import java.io.IOException; import java.sql.SQLException; import java.util.ArrayList; import java.util.List; @@ -36,9 +38,12 @@ import binlogdata.Binlogdata.VEvent; import binlogdata.Binlogdata.VEventType; import binlogdata.Binlogdata.VGtid; +import io.grpc.ChannelCredentials; +import io.grpc.Grpc; +import io.grpc.InsecureChannelCredentials; import io.grpc.ManagedChannel; import io.grpc.ManagedChannelBuilder; - +import io.grpc.TlsChannelCredentials; import io.vitess.proto.Vtgate.VStreamFlags; import io.vitess.proto.Vtgate.VStreamRequest; import io.vitess.proto.grpc.VitessGrpc; @@ -57,6 +62,8 @@ public class VStreamReplicator extends RunLoopProcess implements Replicator { private static final String UPDATE_TYPE = "UPDATE"; private static final String DELETE_TYPE = "DELETE"; + private static final String MAXWELL_USER_AGENT = "Maxwell's Daemon"; + private final MaxwellVitessConfig vitessConfig; private final AbstractProducer producer; private final VitessPosition initPosition; @@ -103,14 +110,11 @@ public VStreamReplicator( } public void startReplicator() throws Exception { - LOGGER.info( - "Starting VStreamReplicator, connecting to Vtgate at {}:{}", + LOGGER.info("Starting VStreamReplicator, connecting to Vtgate at {}:{}", vitessConfig.vtgateHost, vitessConfig.vtgatePort); - this.channel = newChannel( - vitessConfig.vtgateHost, - vitessConfig.vtgatePort, - GRPC_MAX_INBOUND_MESSAGE_SIZE); + this.channel = newChannel(vitessConfig, GRPC_MAX_INBOUND_MESSAGE_SIZE); + VitessGrpc.VitessStub stub = VitessGrpc.newStub(channel); VStreamFlags vStreamFlags = VStreamFlags.newBuilder() @@ -125,12 +129,10 @@ public void startReplicator() throws Exception { .build(); this.responseObserver = new VStreamObserver(queue); - stub.vStream(vstreamRequest, responseObserver); - LOGGER.info("Started VStream"); - this.replicatorStarted = true; + LOGGER.info("Started VStream"); } @Override @@ -425,12 +427,39 @@ public Long getSchemaId() throws SchemaStoreException { } - private static ManagedChannel newChannel(String vtgateHost, int vtgatePort, int maxInboundMessageSize) { - return ManagedChannelBuilder - .forAddress(vtgateHost, vtgatePort) - .usePlaintext() - .maxInboundMessageSize(maxInboundMessageSize) - .keepAliveTime(KEEPALIVE_INTERVAL_SECONDS, TimeUnit.SECONDS) - .build(); + private static ManagedChannel newChannel(MaxwellVitessConfig config, int maxInboundMessageSize) throws IOException { + ChannelCredentials channelCredentials = InsecureChannelCredentials.create(); + + if (!config.usePlaintext) { + TlsChannelCredentials.Builder tlsCredentialsBuilder = TlsChannelCredentials.newBuilder(); + + if (config.tlsCA != null) { + LOGGER.info("Using a custom TLS CA: {}", config.tlsCA); + tlsCredentialsBuilder.trustManager(new File(config.tlsCA)); + } + + // if (config.tlsCert != null && config.tlsKey != null) { + // tlsCredentialsBuilder.keyManager(new File(config.tlsCert), new File(config.tlsKey)); + // } + + channelCredentials = tlsCredentialsBuilder.build(); + } + + ManagedChannelBuilder builder = Grpc.newChannelBuilderForAddress(config.vtgateHost, config.vtgatePort, channelCredentials) + .maxInboundMessageSize(maxInboundMessageSize) + .keepAliveTime(KEEPALIVE_INTERVAL_SECONDS, TimeUnit.SECONDS) + .userAgent(MAXWELL_USER_AGENT); + + if (config.usePlaintext) { + LOGGER.warn("Using plaintext connection to vtgate"); + builder.usePlaintext(); + } + + if (config.tlsServerName != null) { + LOGGER.info("Using TLS server name override: {}", config.tlsServerName); + builder.overrideAuthority(config.tlsServerName); + } + + return builder.build(); } } diff --git a/src/main/java/com/zendesk/maxwell/util/AbstractConfig.java b/src/main/java/com/zendesk/maxwell/util/AbstractConfig.java index d8f5dcdbf..e1c7524b8 100644 --- a/src/main/java/com/zendesk/maxwell/util/AbstractConfig.java +++ b/src/main/java/com/zendesk/maxwell/util/AbstractConfig.java @@ -230,9 +230,15 @@ protected MaxwellVitessConfig parseVitessConfig(OptionSet options, Properties pr MaxwellVitessConfig config = new MaxwellVitessConfig(); config.vtgateHost = fetchStringOption("vitess_host", options, properties, "localhost"); config.vtgatePort = fetchIntegerOption("vitess_port", options, properties, 15991); - config.user = fetchStringOption("vitess_user", options, properties, null); + + config.usePlaintext = fetchBooleanOption("vitess_plaintext", options, properties, true); + config.tlsCA = fetchStringOption("vitess_tls_ca", options, properties, null); + config.tlsServerName = fetchStringOption("vitess_tls_server_name", options, properties, null); + + config.user = fetchStringOption("vitess_user", options, properties, null); config.password = fetchStringOption("vitess_password", options, properties, null); - config.keyspace = fetchStringOption("vitess_keyspace", options, properties, null); + + config.keyspace = fetchStringOption("vitess_keyspace", options, properties, null); config.shard = fetchStringOption("vitess_shard", options, properties, ""); return config; } diff --git a/vitess/config.properties b/vitess/config.properties index 57052e473..efb860a9f 100644 --- a/vitess/config.properties +++ b/vitess/config.properties @@ -12,12 +12,19 @@ producer=stdout vitess=true vitess_keyspace=app_shard vitess_shard= + +# VTgate gRPC address vitess_host=127.0.0.1 -vitess_port=33575 +vitess_port=15991 + +# Use TLS to connect to vtgate +# vitess_plaintext=false +# vitess_tls_ca=/Users/kovyrin/src/github.com/Shopify/vttestserver/tls/vitess.ca.pem +# vitess_tls_server_name=vitess.test # MySQL login info for storing maxwell metadata (a separate vitess keyspace, no sharding) host=127.0.0.1 -port=33577 +port=15306 user=root password= schema_database=maxwell From be4cc0e4b1472a4d35fe00c69faa45040d9c1755 Mon Sep 17 00:00:00 2001 From: Oleksiy Kovyrin Date: Fri, 25 Nov 2022 13:41:30 -0500 Subject: [PATCH 19/32] Added TLS client cert authentication support --- .../java/com/zendesk/maxwell/MaxwellVitessConfig.java | 5 +++++ .../zendesk/maxwell/replication/VStreamReplicator.java | 7 ++++--- .../java/com/zendesk/maxwell/util/AbstractConfig.java | 2 ++ vitess/config.properties | 10 +++++++++- 4 files changed, 20 insertions(+), 4 deletions(-) diff --git a/src/main/java/com/zendesk/maxwell/MaxwellVitessConfig.java b/src/main/java/com/zendesk/maxwell/MaxwellVitessConfig.java index 564937329..fcb93a6f7 100644 --- a/src/main/java/com/zendesk/maxwell/MaxwellVitessConfig.java +++ b/src/main/java/com/zendesk/maxwell/MaxwellVitessConfig.java @@ -12,13 +12,18 @@ public class MaxwellVitessConfig { public boolean usePlaintext; public String tlsCA; + public String tlsCert; + public String tlsKey; public String tlsServerName; public MaxwellVitessConfig() { this.vtgateHost = "localhost"; this.vtgatePort = 15991; this.usePlaintext = true; + this.tlsCA = null; + this.tlsCert = null; + this.tlsKey = null; this.tlsServerName = null; this.user = null; diff --git a/src/main/java/com/zendesk/maxwell/replication/VStreamReplicator.java b/src/main/java/com/zendesk/maxwell/replication/VStreamReplicator.java index 2a22ace87..76a27d8e1 100644 --- a/src/main/java/com/zendesk/maxwell/replication/VStreamReplicator.java +++ b/src/main/java/com/zendesk/maxwell/replication/VStreamReplicator.java @@ -438,9 +438,10 @@ private static ManagedChannel newChannel(MaxwellVitessConfig config, int maxInbo tlsCredentialsBuilder.trustManager(new File(config.tlsCA)); } - // if (config.tlsCert != null && config.tlsKey != null) { - // tlsCredentialsBuilder.keyManager(new File(config.tlsCert), new File(config.tlsKey)); - // } + if (config.tlsCert != null && config.tlsKey != null) { + LOGGER.info("TLS client credentials: cert={}, key={}", config.tlsCert, config.tlsKey); + tlsCredentialsBuilder.keyManager(new File(config.tlsCert), new File(config.tlsKey)); + } channelCredentials = tlsCredentialsBuilder.build(); } diff --git a/src/main/java/com/zendesk/maxwell/util/AbstractConfig.java b/src/main/java/com/zendesk/maxwell/util/AbstractConfig.java index e1c7524b8..8d6dcbb0b 100644 --- a/src/main/java/com/zendesk/maxwell/util/AbstractConfig.java +++ b/src/main/java/com/zendesk/maxwell/util/AbstractConfig.java @@ -233,6 +233,8 @@ protected MaxwellVitessConfig parseVitessConfig(OptionSet options, Properties pr config.usePlaintext = fetchBooleanOption("vitess_plaintext", options, properties, true); config.tlsCA = fetchStringOption("vitess_tls_ca", options, properties, null); + config.tlsCert = fetchStringOption("vitess_tls_cert", options, properties, null); + config.tlsKey = fetchStringOption("vitess_tls_key", options, properties, null); config.tlsServerName = fetchStringOption("vitess_tls_server_name", options, properties, null); config.user = fetchStringOption("vitess_user", options, properties, null); diff --git a/vitess/config.properties b/vitess/config.properties index efb860a9f..b1b70cda6 100644 --- a/vitess/config.properties +++ b/vitess/config.properties @@ -19,9 +19,17 @@ vitess_port=15991 # Use TLS to connect to vtgate # vitess_plaintext=false -# vitess_tls_ca=/Users/kovyrin/src/github.com/Shopify/vttestserver/tls/vitess.ca.pem + +# Override the TLS server name used for server certificate verification (default: vitess_host) # vitess_tls_server_name=vitess.test +# Use a custom TLS CA certificate to verify the server certificate +# vitess_tls_ca=/Users/kovyrin/src/github.com/Shopify/vttestserver/tls/vitess.ca.pem + +# Use a client TLS certificate to authenticate to vtgate +# vitess_tls_cert=/Users/kovyrin/src/github.com/Shopify/vttestserver/tls/client.pem +# vitess_tls_key=/Users/kovyrin/src/github.com/Shopify/vttestserver/tls/client.key + # MySQL login info for storing maxwell metadata (a separate vitess keyspace, no sharding) host=127.0.0.1 port=15306 From 2eabf29f9761967bce1909d5a1887809f7d43763 Mon Sep 17 00:00:00 2001 From: Oleksiy Kovyrin Date: Sat, 26 Nov 2022 10:10:51 -0500 Subject: [PATCH 20/32] Revert unrelated changes to Position class --- .../zendesk/maxwell/replication/Position.java | 18 +++++++++--------- 1 file changed, 9 insertions(+), 9 deletions(-) diff --git a/src/main/java/com/zendesk/maxwell/replication/Position.java b/src/main/java/com/zendesk/maxwell/replication/Position.java index f5c3f0af5..3ed0a4c78 100644 --- a/src/main/java/com/zendesk/maxwell/replication/Position.java +++ b/src/main/java/com/zendesk/maxwell/replication/Position.java @@ -54,11 +54,13 @@ public String toCommandline() { @Override public boolean equals(Object o) { - if (o instanceof Position) { - Position other = (Position) o; - return lastHeartbeatRead == other.lastHeartbeatRead && binlogPosition.equals(other.binlogPosition); + if ( !(o instanceof Position) ) { + return false; } - return false; + Position other = (Position) o; + + return lastHeartbeatRead == other.lastHeartbeatRead + && binlogPosition.equals(other.binlogPosition); } @Override @@ -67,10 +69,8 @@ public int hashCode() { } public boolean newerThan(Position other) { - if (other instanceof Position) { - Position mOther = (Position) other; - return binlogPosition.newerThan(mOther.binlogPosition); - } - return true; + if ( other == null ) + return true; + return this.getBinlogPosition().newerThan(other.getBinlogPosition()); } } From c01b8a5754e6a10373661354686b4ebe5f71e638 Mon Sep 17 00:00:00 2001 From: Oleksiy Kovyrin Date: Wed, 30 Nov 2022 14:08:26 -0500 Subject: [PATCH 21/32] Ignore an empty VGTID from the positions table --- .../com/zendesk/maxwell/schema/VitessPositionStore.java | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/src/main/java/com/zendesk/maxwell/schema/VitessPositionStore.java b/src/main/java/com/zendesk/maxwell/schema/VitessPositionStore.java index 5882a50e4..d74648060 100644 --- a/src/main/java/com/zendesk/maxwell/schema/VitessPositionStore.java +++ b/src/main/java/com/zendesk/maxwell/schema/VitessPositionStore.java @@ -66,6 +66,12 @@ protected Position positionFromResultSet(ResultSet rs) throws SQLException { } String vgtidString = rs.getString("vitess_gtid"); + if (vgtidString == null) { + LOGGER.warn("Read a null VTGID value from the positions table, assuming we need to start from the current position"); + return null; + } + + LOGGER.debug("Read VGTID from positions: {}", vgtidString); Vgtid vgtid = Vgtid.of(vgtidString); return new VitessPosition(vgtid); From 9c35476497deb2dbf45503a1a021bcff34dca371 Mon Sep 17 00:00:00 2001 From: Oleksiy Kovyrin Date: Wed, 30 Nov 2022 14:09:10 -0500 Subject: [PATCH 22/32] Restore ports --- vitess/config.properties | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/vitess/config.properties b/vitess/config.properties index b1b70cda6..609636c0e 100644 --- a/vitess/config.properties +++ b/vitess/config.properties @@ -15,7 +15,7 @@ vitess_shard= # VTgate gRPC address vitess_host=127.0.0.1 -vitess_port=15991 +vitess_port=33575 # Use TLS to connect to vtgate # vitess_plaintext=false @@ -32,7 +32,7 @@ vitess_port=15991 # MySQL login info for storing maxwell metadata (a separate vitess keyspace, no sharding) host=127.0.0.1 -port=15306 +port=33577 user=root password= schema_database=maxwell From ea115c66d538270ce1f811e0f6aed3ba0d421293 Mon Sep 17 00:00:00 2001 From: Oleksiy Kovyrin Date: Wed, 30 Nov 2022 16:58:14 -0500 Subject: [PATCH 23/32] Ensure we get a PKCS#8 private key --- .../replication/VStreamReplicator.java | 19 +++++++++++++++++++ 1 file changed, 19 insertions(+) diff --git a/src/main/java/com/zendesk/maxwell/replication/VStreamReplicator.java b/src/main/java/com/zendesk/maxwell/replication/VStreamReplicator.java index 76a27d8e1..e90f95c60 100644 --- a/src/main/java/com/zendesk/maxwell/replication/VStreamReplicator.java +++ b/src/main/java/com/zendesk/maxwell/replication/VStreamReplicator.java @@ -2,6 +2,10 @@ import java.io.File; import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.Paths; import java.sql.SQLException; import java.util.ArrayList; import java.util.List; @@ -440,6 +444,7 @@ private static ManagedChannel newChannel(MaxwellVitessConfig config, int maxInbo if (config.tlsCert != null && config.tlsKey != null) { LOGGER.info("TLS client credentials: cert={}, key={}", config.tlsCert, config.tlsKey); + ensurePkcs8(config.tlsKey); tlsCredentialsBuilder.keyManager(new File(config.tlsCert), new File(config.tlsKey)); } @@ -463,4 +468,18 @@ private static ManagedChannel newChannel(MaxwellVitessConfig config, int maxInbo return builder.build(); } + + // Makes sure the given private key file is in PKCS#8 format. + private static void ensurePkcs8(String keyFile) { + try { + Path path = Paths.get(keyFile); + String keyContent = new String(Files.readAllBytes(path), StandardCharsets.UTF_8); + if (!keyContent.contains("BEGIN PRIVATE KEY")) { + LOGGER.error("Private key file {} is not in PKCS#8 format, please convert it", keyFile); + throw new RuntimeException("Private key file is not in PKCS#8 format"); + } + } catch (IOException e) { + throw new RuntimeException("Failed to read private key file: " + keyFile, e); + } + } } From bb3370cdd624ecc6c36adb453995233d17c20986 Mon Sep 17 00:00:00 2001 From: Oleksiy Kovyrin Date: Wed, 30 Nov 2022 17:50:53 -0500 Subject: [PATCH 24/32] No need to explicitly enable plaintext --- .../java/com/zendesk/maxwell/replication/VStreamReplicator.java | 1 - 1 file changed, 1 deletion(-) diff --git a/src/main/java/com/zendesk/maxwell/replication/VStreamReplicator.java b/src/main/java/com/zendesk/maxwell/replication/VStreamReplicator.java index e90f95c60..4171c84b0 100644 --- a/src/main/java/com/zendesk/maxwell/replication/VStreamReplicator.java +++ b/src/main/java/com/zendesk/maxwell/replication/VStreamReplicator.java @@ -458,7 +458,6 @@ private static ManagedChannel newChannel(MaxwellVitessConfig config, int maxInbo if (config.usePlaintext) { LOGGER.warn("Using plaintext connection to vtgate"); - builder.usePlaintext(); } if (config.tlsServerName != null) { From 96a8004c191b7ae3be285ac32ddf36db1fba495e Mon Sep 17 00:00:00 2001 From: Oleksiy Kovyrin Date: Mon, 5 Dec 2022 16:25:35 -0500 Subject: [PATCH 25/32] Implement user/password auth on grpc calls --- .../com/zendesk/maxwell/replication/VStreamReplicator.java | 7 +++++++ vitess/config.properties | 4 ++++ 2 files changed, 11 insertions(+) diff --git a/src/main/java/com/zendesk/maxwell/replication/VStreamReplicator.java b/src/main/java/com/zendesk/maxwell/replication/VStreamReplicator.java index 4171c84b0..e67205e91 100644 --- a/src/main/java/com/zendesk/maxwell/replication/VStreamReplicator.java +++ b/src/main/java/com/zendesk/maxwell/replication/VStreamReplicator.java @@ -51,6 +51,7 @@ import io.vitess.proto.Vtgate.VStreamFlags; import io.vitess.proto.Vtgate.VStreamRequest; import io.vitess.proto.grpc.VitessGrpc; +import io.vitess.client.grpc.StaticAuthCredentials; import io.vitess.proto.Topodata; public class VStreamReplicator extends RunLoopProcess implements Replicator { @@ -120,6 +121,12 @@ public void startReplicator() throws Exception { this.channel = newChannel(vitessConfig, GRPC_MAX_INBOUND_MESSAGE_SIZE); VitessGrpc.VitessStub stub = VitessGrpc.newStub(channel); + if (vitessConfig.user != null && vitessConfig.password != null) { + LOGGER.info("Using provided credentials for Vtgate grpc calls"); + stub = stub.withCallCredentials( + new StaticAuthCredentials(vitessConfig.user, vitessConfig.password) + ); + } VStreamFlags vStreamFlags = VStreamFlags.newBuilder() .setStopOnReshard(true) diff --git a/vitess/config.properties b/vitess/config.properties index 609636c0e..b5cd772e8 100644 --- a/vitess/config.properties +++ b/vitess/config.properties @@ -17,6 +17,10 @@ vitess_shard= vitess_host=127.0.0.1 vitess_port=33575 +# Credentials to use for Vitess gRPC calls +# vitess_user=banana +# vitess_password=ban@na + # Use TLS to connect to vtgate # vitess_plaintext=false From 92d58f7c9c6e9eb33cec43cdf2e683ae29a7bff7 Mon Sep 17 00:00:00 2001 From: Oleksiy Kovyrin Date: Wed, 7 Dec 2022 13:19:22 -0500 Subject: [PATCH 26/32] Tell the server we want to stop before closing the stream --- .../java/com/zendesk/maxwell/replication/VStreamReplicator.java | 1 + 1 file changed, 1 insertion(+) diff --git a/src/main/java/com/zendesk/maxwell/replication/VStreamReplicator.java b/src/main/java/com/zendesk/maxwell/replication/VStreamReplicator.java index e67205e91..ac62c64cf 100644 --- a/src/main/java/com/zendesk/maxwell/replication/VStreamReplicator.java +++ b/src/main/java/com/zendesk/maxwell/replication/VStreamReplicator.java @@ -149,6 +149,7 @@ public void startReplicator() throws Exception { @Override protected void beforeStop() throws Exception { responseObserver.stop(); + responseObserver.onCompleted(); channel.shutdown(); channel.awaitTermination(500, TimeUnit.MILLISECONDS); From 7d8d528ec0d98dd418a25a907c1b08a50ef78ea7 Mon Sep 17 00:00:00 2001 From: Oleksiy Kovyrin Date: Thu, 8 Dec 2022 10:23:59 -0500 Subject: [PATCH 27/32] Re-enable opencensus stuff --- pom.xml | 7 +++--- .../maxwell/monitoring/MaxwellMetrics.java | 25 +++++++++---------- 2 files changed, 15 insertions(+), 17 deletions(-) diff --git a/pom.xml b/pom.xml index e1a24e412..570032f1b 100644 --- a/pom.xml +++ b/pom.xml @@ -297,7 +297,7 @@ jedis 3.5.1 - + io.dropwizard.metrics @@ -339,9 +339,8 @@ simpleclient_servlet 0.9.0 - - + diff --git a/src/main/java/com/zendesk/maxwell/monitoring/MaxwellMetrics.java b/src/main/java/com/zendesk/maxwell/monitoring/MaxwellMetrics.java index 9adc374b6..65c04c90d 100644 --- a/src/main/java/com/zendesk/maxwell/monitoring/MaxwellMetrics.java +++ b/src/main/java/com/zendesk/maxwell/monitoring/MaxwellMetrics.java @@ -12,7 +12,7 @@ import com.zendesk.maxwell.MaxwellContext; import io.prometheus.client.CollectorRegistry; import io.prometheus.client.dropwizard.DropwizardExports; -// import io.opencensus.exporter.stats.stackdriver.StackdriverStatsExporter; +import io.opencensus.exporter.stats.stackdriver.StackdriverStatsExporter; import org.apache.commons.lang.StringUtils; import org.slf4j.Logger; @@ -126,18 +126,17 @@ private void setup(MaxwellConfig config) { } if (config.metricsReportingType.contains(reportingTypeStackdriver)) { - throw new RuntimeException("Stackdriver metrics reporting is not supported for now"); - // io.opencensus.metrics.Metrics.getExportComponent().getMetricProducerManager().add( - // new io.opencensus.contrib.dropwizard.DropWizardMetrics( - // Collections.singletonList(this.registry))); - - // try { - // StackdriverStatsExporter.createAndRegister(); - // } catch (java.io.IOException e) { - // LOGGER.error("Maxwell encountered an error in creating the stackdriver exporter.", e); - // } - - // LOGGER.info("Stackdriver metrics reporter enabled"); + io.opencensus.metrics.Metrics.getExportComponent().getMetricProducerManager().add( + new io.opencensus.contrib.dropwizard.DropWizardMetrics( + Collections.singletonList(this.registry))); + + try { + StackdriverStatsExporter.createAndRegister(); + } catch (java.io.IOException e) { + LOGGER.error("Maxwell encountered an error in creating the stackdriver exporter.", e); + } + + LOGGER.info("Stackdriver metrics reporter enabled"); } if (config.metricsReportingType.contains(reportingTypeHttp)) { From 64f02d19b0d0c7489ffb445e1dabd3db21dcb319 Mon Sep 17 00:00:00 2001 From: Oleksiy Kovyrin Date: Thu, 8 Dec 2022 10:25:59 -0500 Subject: [PATCH 28/32] Remove the extra empty line --- pom.xml | 1 - 1 file changed, 1 deletion(-) diff --git a/pom.xml b/pom.xml index 570032f1b..36d2e9c73 100644 --- a/pom.xml +++ b/pom.xml @@ -297,7 +297,6 @@ jedis 3.5.1 - io.dropwizard.metrics From e96ef9c1548f3187211c7f4ea5bf31c5ef2ab1e2 Mon Sep 17 00:00:00 2001 From: Oleksiy Kovyrin Date: Mon, 12 Dec 2022 10:57:22 -0500 Subject: [PATCH 29/32] Capture old values from VStream UPDATE events --- .../maxwell/replication/VStreamReplicator.java | 17 +++++++++++++++-- .../maxwell/replication/vitess/VitessTable.java | 9 +++++---- 2 files changed, 20 insertions(+), 6 deletions(-) diff --git a/src/main/java/com/zendesk/maxwell/replication/VStreamReplicator.java b/src/main/java/com/zendesk/maxwell/replication/VStreamReplicator.java index ac62c64cf..4dbdd7346 100644 --- a/src/main/java/com/zendesk/maxwell/replication/VStreamReplicator.java +++ b/src/main/java/com/zendesk/maxwell/replication/VStreamReplicator.java @@ -53,6 +53,7 @@ import io.vitess.proto.grpc.VitessGrpc; import io.vitess.client.grpc.StaticAuthCredentials; import io.vitess.proto.Topodata; +import io.vitess.proto.Query.Row; public class VStreamReplicator extends RunLoopProcess implements Replicator { private static final Logger LOGGER = LoggerFactory.getLogger(VStreamReplicator.class); @@ -371,7 +372,6 @@ private List rowEventToMaps(VEvent event, long xid) { LOGGER.debug("Filtering out event for table {}.{}", table.getSchemaName(), table.getTableName()); continue; } - List columns = table.resolveColumns(rowChange); RowMap rowMap = new RowMap( changeType, @@ -386,10 +386,23 @@ private List rowEventToMaps(VEvent event, long xid) { rowMap.setXid(xid); - for (ReplicationMessageColumn column : columns) { + // Copy column values to the row map, use the new values when available, otherwise use the old ones + + Row row = rowChange.hasAfter() ? rowChange.getAfter() : rowChange.getBefore(); + List afterColumns = table.resolveColumnsFromRow(row); + for (ReplicationMessageColumn column : afterColumns) { rowMap.putData(column.getName(), column.getValue()); } + // Copy old values to the row map for cases when we have both the old and the new values + if (changeType.equals(UPDATE_TYPE)) { + Row beforeRow = rowChange.getBefore(); + List beforeColumns = table.resolveColumnsFromRow(beforeRow); + for (ReplicationMessageColumn column : beforeColumns) { + rowMap.putOldData(column.getName(), column.getValue()); + } + } + rowMaps.add(rowMap); } diff --git a/src/main/java/com/zendesk/maxwell/replication/vitess/VitessTable.java b/src/main/java/com/zendesk/maxwell/replication/vitess/VitessTable.java index 08a0b381e..861fbf3aa 100644 --- a/src/main/java/com/zendesk/maxwell/replication/vitess/VitessTable.java +++ b/src/main/java/com/zendesk/maxwell/replication/vitess/VitessTable.java @@ -50,11 +50,10 @@ public String toString() { } /** - * Resolve the vEvent data to a list of replication message columns (with - * values). + * Resolve a specific row from vEvent data to a list of replication message + * columns (with values). */ - public List resolveColumns(RowChange rowChange) { - Row row = rowChange.hasAfter() ? rowChange.getAfter() : rowChange.getBefore(); + public List resolveColumnsFromRow(Row row) { int changedColumnsCnt = row.getLengthsCount(); if (columns.size() != changedColumnsCnt) { throw new IllegalStateException( @@ -86,4 +85,6 @@ public List resolveColumns(RowChange rowChange) { } return eventColumns; } + + } From 312aab272b3c9a9724fcd1a61b3a134c5c314301 Mon Sep 17 00:00:00 2001 From: Oleksiy Kovyrin Date: Mon, 12 Dec 2022 10:57:38 -0500 Subject: [PATCH 30/32] Change vitess ports to not conflict with Vitess dev clusters --- vitess/config.properties | 4 ++-- vitess/docker-compose.yml | 6 +++--- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/vitess/config.properties b/vitess/config.properties index b5cd772e8..cc941d739 100644 --- a/vitess/config.properties +++ b/vitess/config.properties @@ -15,7 +15,7 @@ vitess_shard= # VTgate gRPC address vitess_host=127.0.0.1 -vitess_port=33575 +vitess_port=43575 # Credentials to use for Vitess gRPC calls # vitess_user=banana @@ -36,7 +36,7 @@ vitess_port=33575 # MySQL login info for storing maxwell metadata (a separate vitess keyspace, no sharding) host=127.0.0.1 -port=33577 +port=43577 user=root password= schema_database=maxwell diff --git a/vitess/docker-compose.yml b/vitess/docker-compose.yml index 7c14ed4d3..07ff6e5a1 100644 --- a/vitess/docker-compose.yml +++ b/vitess/docker-compose.yml @@ -3,8 +3,8 @@ services: vttestserver: image: vitess/vttestserver:mysql57 ports: - - "33575:33575" # gRPC (VTGate, etc) - - "33577:33577" # MySQL + - "43575:33575" # gRPC (VTGate, etc) + - "43577:33577" # MySQL environment: - PORT=33574 - PLANNER_VERSION=gen4fallback @@ -13,7 +13,7 @@ services: - MYSQL_MAX_CONNECTIONS=70000 - MYSQL_BIND_HOST=0.0.0.0 healthcheck: - test: ["CMD", "mysqladmin", "ping", "-h127.0.0.1", "-P33577"] + test: ["CMD", "mysqladmin", "ping", "-h127.0.0.1", "-P43577"] interval: 5s timeout: 2s retries: 5 From be63881bd669da34cf88561f1d5040ce66a6a7fe Mon Sep 17 00:00:00 2001 From: Oleksiy Kovyrin Date: Wed, 1 Feb 2023 09:38:42 -0500 Subject: [PATCH 31/32] Do not recover from positions with different client ids when running on Vitess since they may be following a different keyspace/shards --- .../com/zendesk/maxwell/schema/VitessPositionStore.java | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/src/main/java/com/zendesk/maxwell/schema/VitessPositionStore.java b/src/main/java/com/zendesk/maxwell/schema/VitessPositionStore.java index d74648060..9e558a8ef 100644 --- a/src/main/java/com/zendesk/maxwell/schema/VitessPositionStore.java +++ b/src/main/java/com/zendesk/maxwell/schema/VitessPositionStore.java @@ -53,6 +53,12 @@ public void set(Position p) throws SQLException, DuplicateProcessException { }); } + @Override + public Position getLatestFromAnyClient() throws SQLException { + return null; // Never recover from a different client on Vitess + } + + @Override public long heartbeat() throws Exception { // Heartbeats are not supported in Vitess. From ccf2446c6d948016b792971b5240a4586eb8e318 Mon Sep 17 00:00:00 2001 From: Oleksiy Kovyrin Date: Fri, 24 Mar 2023 17:54:38 -0400 Subject: [PATCH 32/32] Move extremely verbose Vitess replicator logs to the trace level --- .../maxwell/replication/VStreamObserver.java | 2 +- .../maxwell/replication/VStreamReplicator.java | 13 +++++++++---- 2 files changed, 10 insertions(+), 5 deletions(-) diff --git a/src/main/java/com/zendesk/maxwell/replication/VStreamObserver.java b/src/main/java/com/zendesk/maxwell/replication/VStreamObserver.java index 317a03532..96fa5412c 100644 --- a/src/main/java/com/zendesk/maxwell/replication/VStreamObserver.java +++ b/src/main/java/com/zendesk/maxwell/replication/VStreamObserver.java @@ -38,7 +38,7 @@ public void onNext(Vtgate.VStreamResponse response) { List messageEvents = response.getEventsList(); for (VEvent event : messageEvents) { - LOGGER.debug("VEvent: {}", event); + LOGGER.trace("VEvent: {}", event); enqueueEvent(event); } } diff --git a/src/main/java/com/zendesk/maxwell/replication/VStreamReplicator.java b/src/main/java/com/zendesk/maxwell/replication/VStreamReplicator.java index 4dbdd7346..36d723682 100644 --- a/src/main/java/com/zendesk/maxwell/replication/VStreamReplicator.java +++ b/src/main/java/com/zendesk/maxwell/replication/VStreamReplicator.java @@ -169,7 +169,9 @@ public void work() throws Exception { try { row = getRow(); } catch (InterruptedException e) { - LOGGER.debug("Interrupted while waiting for row"); + if (LOGGER.isTraceEnabled()) { + LOGGER.trace("Interrupted while waiting for row"); + } } if (row == null) { @@ -271,7 +273,7 @@ private RowMapBuffer getTransactionRows(VEvent beginEvent) throws Exception { // Since transactions in VStream do not have an XID value, we generate one long xid = System.currentTimeMillis() * 1000 + Math.abs(beginEvent.hashCode()) % 1000; - LOGGER.debug("Generated transaction id: {}", xid); + LOGGER.trace("Generated transaction id: {}", xid); buffer.setXid(xid); // Since specific VStream events do not provide the VGTID, we capture it from @@ -292,7 +294,7 @@ private RowMapBuffer getTransactionRows(VEvent beginEvent) throws Exception { } if (eventType == VEventType.COMMIT) { - LOGGER.debug("Received COMMIT event"); + LOGGER.trace("Received COMMIT event"); if (!buffer.isEmpty()) { // Set TX flag and the position on the last row in the transaction RowMap lastEvent = buffer.getLast(); @@ -369,7 +371,10 @@ private List rowEventToMaps(VEvent event, long xid) { String changeType = rowChangeToMaxwellType(rowChange); final VitessTable table = resolveTable(qualifiedTableName); if (!filter.includes(table.getSchemaName(), table.getTableName())) { - LOGGER.debug("Filtering out event for table {}.{}", table.getSchemaName(), table.getTableName()); + if (LOGGER.isTraceEnabled()) { + LOGGER.trace("Filtering out event for table {}.{}", + table.getSchemaName(), table.getTableName()); + } continue; }