DBZ-5206 Remove the deprecated monitored tables metrics
This commit is contained in:
parent
327e7fd7b3
commit
acb743bd34
@ -97,7 +97,7 @@ public void testSnapshotOnlyMetrics() throws Exception {
|
||||
assertThat(mBeanServer.getAttribute(objectName, "TotalNumberOfEventsSeen")).isEqualTo(6L);
|
||||
assertThat(mBeanServer.getAttribute(objectName, "NumberOfEventsFiltered")).isEqualTo(0L);
|
||||
assertThat(mBeanServer.getAttribute(objectName, "NumberOfErroneousEvents")).isEqualTo(0L);
|
||||
assertThat(mBeanServer.getAttribute(objectName, "MonitoredTables")).isEqualTo(new String[]{ "rs0.dbit.restaurants" });
|
||||
assertThat(mBeanServer.getAttribute(objectName, "CapturedTables")).isEqualTo(new String[]{ "rs0.dbit.restaurants" });
|
||||
assertThat(mBeanServer.getAttribute(objectName, "LastEvent")).isNotNull();
|
||||
assertThat(mBeanServer.getAttribute(objectName, "NumberOfDisconnects")).isEqualTo(0L);
|
||||
}
|
||||
@ -131,7 +131,7 @@ public void testStreamingOnlyMetrics() throws Exception {
|
||||
|
||||
assertThat(mBeanServer.getAttribute(objectName, "SourceEventPosition")).isNotNull();
|
||||
assertThat(mBeanServer.getAttribute(objectName, "Connected")).isEqualTo(true);
|
||||
assertThat(mBeanServer.getAttribute(objectName, "MonitoredTables")).isEqualTo(new String[]{});
|
||||
assertThat(mBeanServer.getAttribute(objectName, "CapturedTables")).isEqualTo(new String[]{});
|
||||
assertThat(mBeanServer.getAttribute(objectName, "LastEvent")).isNotNull();
|
||||
assertThat(mBeanServer.getAttribute(objectName, "TotalNumberOfEventsSeen")).isEqualTo(6L);
|
||||
assertThat(mBeanServer.getAttribute(objectName, "NumberOfEventsFiltered")).isEqualTo(0L);
|
||||
|
@ -153,11 +153,6 @@ public void setMilliSecondsBehindSource(long value) {
|
||||
milliSecondsBehindMaster.set(value);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String[] getMonitoredTables() {
|
||||
return schema.capturedTablesAsStringArray();
|
||||
}
|
||||
|
||||
@Override
|
||||
public String[] getCapturedTables() {
|
||||
return schema.capturedTablesAsStringArray();
|
||||
|
@ -221,7 +221,7 @@ private void assertSnapshotMetrics() throws Exception {
|
||||
|
||||
// Check snapshot metrics
|
||||
assertThat(mBeanServer.getAttribute(getSnapshotMetricsObjectName(), "TotalTableCount")).isEqualTo(1);
|
||||
assertThat(mBeanServer.getAttribute(getSnapshotMetricsObjectName(), "MonitoredTables"))
|
||||
assertThat(mBeanServer.getAttribute(getSnapshotMetricsObjectName(), "CapturedTables"))
|
||||
.isEqualTo(new String[]{ DATABASE.qualifiedTableName("simple") });
|
||||
assertThat(mBeanServer.getAttribute(getSnapshotMetricsObjectName(), "TotalNumberOfEventsSeen")).isEqualTo(2L);
|
||||
assertThat(mBeanServer.getAttribute(getSnapshotMetricsObjectName(), "NumberOfEventsFiltered")).isEqualTo(0L);
|
||||
@ -258,8 +258,8 @@ private void assertStreamingMetrics(long events) throws Exception {
|
||||
.isGreaterThanOrEqualTo(events);
|
||||
|
||||
Awaitility.await().atMost(Duration.ofMinutes(1)).until(() -> ((String[]) mBeanServer
|
||||
.getAttribute(getStreamingMetricsObjectName(), "MonitoredTables")).length > 0);
|
||||
assertThat(mBeanServer.getAttribute(getStreamingMetricsObjectName(), "MonitoredTables"))
|
||||
.getAttribute(getStreamingMetricsObjectName(), "CapturedTables")).length > 0);
|
||||
assertThat(mBeanServer.getAttribute(getStreamingMetricsObjectName(), "CapturedTables"))
|
||||
.isEqualTo(new String[]{ DATABASE.qualifiedTableName("simple") });
|
||||
}
|
||||
|
||||
|
@ -2139,14 +2139,14 @@ public void shouldNotIssueWarningForNoMonitoredTablesAfterApplyingFilters() thro
|
||||
TestHelper.execute(INSERT_STMT);
|
||||
Configuration config = TestHelper.defaultConfig().with(PostgresConnectorConfig.SCHEMA_INCLUDE_LIST, "s2").build();
|
||||
|
||||
// Start connector, verify that it does not log no monitored tables warning
|
||||
// Start connector, verify that it does not log no captured tables warning
|
||||
start(PostgresConnector.class, config);
|
||||
waitForSnapshotToBeCompleted();
|
||||
SourceRecords records = consumeRecordsByTopic(1);
|
||||
assertThat(logInterceptor.containsMessage(DatabaseSchema.NO_CAPTURED_DATA_COLLECTIONS_WARNING)).isFalse();
|
||||
stopConnector();
|
||||
|
||||
// Restart connector, verify it does not log no monitored tables warning
|
||||
// Restart connector, verify it does not log no captured tables warning
|
||||
start(PostgresConnector.class, config);
|
||||
waitForStreamingRunning();
|
||||
assertThat(logInterceptor.containsMessage(DatabaseSchema.NO_CAPTURED_DATA_COLLECTIONS_WARNING)).isFalse();
|
||||
|
@ -148,7 +148,7 @@ private void assertSnapshotMetrics() throws Exception {
|
||||
|
||||
// Check snapshot metrics
|
||||
Assertions.assertThat(mBeanServer.getAttribute(getSnapshotMetricsObjectName(), "TotalTableCount")).isEqualTo(1);
|
||||
Assertions.assertThat(mBeanServer.getAttribute(getSnapshotMetricsObjectName(), "MonitoredTables")).isEqualTo(new String[]{ "public.simple" });
|
||||
Assertions.assertThat(mBeanServer.getAttribute(getSnapshotMetricsObjectName(), "CapturedTables")).isEqualTo(new String[]{ "public.simple" });
|
||||
Assertions.assertThat(mBeanServer.getAttribute(getSnapshotMetricsObjectName(), "TotalNumberOfEventsSeen")).isEqualTo(2L);
|
||||
Assertions.assertThat(mBeanServer.getAttribute(getSnapshotMetricsObjectName(), "RemainingTableCount")).isEqualTo(0);
|
||||
Assertions.assertThat(mBeanServer.getAttribute(getSnapshotMetricsObjectName(), "SnapshotRunning")).isEqualTo(false);
|
||||
@ -171,7 +171,7 @@ private void assertSnapshotNotExecutedMetrics() throws Exception {
|
||||
|
||||
// Check snapshot metrics
|
||||
Assertions.assertThat(mBeanServer.getAttribute(getSnapshotMetricsObjectName(), "TotalTableCount")).isEqualTo(0);
|
||||
Assertions.assertThat(mBeanServer.getAttribute(getSnapshotMetricsObjectName(), "MonitoredTables")).isEqualTo(new String[]{});
|
||||
Assertions.assertThat(mBeanServer.getAttribute(getSnapshotMetricsObjectName(), "CapturedTables")).isEqualTo(new String[]{});
|
||||
Assertions.assertThat(mBeanServer.getAttribute(getSnapshotMetricsObjectName(), "TotalNumberOfEventsSeen")).isEqualTo(0L);
|
||||
Assertions.assertThat(mBeanServer.getAttribute(getSnapshotMetricsObjectName(), "RemainingTableCount")).isEqualTo(0);
|
||||
Assertions.assertThat(mBeanServer.getAttribute(getSnapshotMetricsObjectName(), "SnapshotRunning")).isEqualTo(false);
|
||||
@ -196,7 +196,7 @@ private void assertStreamingMetrics() throws Exception {
|
||||
Assertions.assertThat(mBeanServer.getAttribute(getStreamingMetricsObjectName(), "Connected")).isEqualTo(true);
|
||||
Assertions.assertThat(mBeanServer.getAttribute(getStreamingMetricsObjectName(), "TotalNumberOfEventsSeen")).isEqualTo(2L);
|
||||
// todo: this does not seem to be populated?
|
||||
// Assertions.assertThat(mBeanServer.getAttribute(getStreamingMetricsObjectName(), "MonitoredTables")).isEqualTo(new String[] {"public.simple"});
|
||||
// Assertions.assertThat(mBeanServer.getAttribute(getStreamingMetricsObjectName(), "CapturedTables")).isEqualTo(new String[] {"public.simple"});
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -317,11 +317,11 @@ public static SnapshotIsolationMode parse(String value, String defaultValue) {
|
||||
.withGroup(Field.createGroupEntry(Field.Group.CONNECTOR_SNAPSHOT, 1))
|
||||
.withWidth(Width.SHORT)
|
||||
.withImportance(Importance.LOW)
|
||||
.withDescription("Controls which transaction isolation level is used and how long the connector locks the monitored tables. "
|
||||
.withDescription("Controls which transaction isolation level is used and how long the connector locks the captured tables. "
|
||||
+ "The default is '" + SnapshotIsolationMode.REPEATABLE_READ.getValue()
|
||||
+ "', which means that repeatable read isolation level is used. In addition, exclusive locks are taken only during schema snapshot. "
|
||||
+ "Using a value of '" + SnapshotIsolationMode.EXCLUSIVE.getValue()
|
||||
+ "' ensures that the connector holds the exclusive lock (and thus prevents any reads and updates) for all monitored tables during the entire snapshot duration. "
|
||||
+ "' ensures that the connector holds the exclusive lock (and thus prevents any reads and updates) for all captured tables during the entire snapshot duration. "
|
||||
+ "When '" + SnapshotIsolationMode.SNAPSHOT.getValue()
|
||||
+ "' is specified, connector runs the initial snapshot in SNAPSHOT isolation level, which guarantees snapshot consistency. In addition, neither table nor row-level locks are held. "
|
||||
+ "When '" + SnapshotIsolationMode.READ_COMMITTED.getValue()
|
||||
|
@ -17,12 +17,5 @@
|
||||
public interface ChangeEventSourceMetricsMXBean extends CommonEventMetricsMXBean, QueueMetricsMXBean,
|
||||
SchemaMetricsMXBean {
|
||||
|
||||
/**
|
||||
* @deprecated Superseded by the 'Captured Tables' metric. Use {@link #getCapturedTables()}.
|
||||
* Scheduled for removal in a future release.
|
||||
*/
|
||||
@Deprecated
|
||||
String[] getMonitoredTables();
|
||||
|
||||
void reset();
|
||||
}
|
||||
|
@ -70,16 +70,6 @@ public long getSnapshotDurationInSeconds() {
|
||||
return snapshotMeter.getSnapshotDurationInSeconds();
|
||||
}
|
||||
|
||||
/**
|
||||
* @deprecated Superseded by the 'Captured Tables' metric. Use {@link #getCapturedTables()}.
|
||||
* Scheduled for removal in a future release.
|
||||
*/
|
||||
@Override
|
||||
@Deprecated
|
||||
public String[] getMonitoredTables() {
|
||||
return snapshotMeter.getCapturedTables();
|
||||
}
|
||||
|
||||
@Override
|
||||
public String[] getCapturedTables() {
|
||||
return snapshotMeter.getCapturedTables();
|
||||
|
@ -52,16 +52,6 @@ public boolean isConnected() {
|
||||
return connectionMeter.isConnected();
|
||||
}
|
||||
|
||||
/**
|
||||
* @deprecated Superseded by the 'Captured Tables' metric. Use {@link #getCapturedTables()}.
|
||||
* Scheduled for removal in a future release.
|
||||
*/
|
||||
@Override
|
||||
@Deprecated
|
||||
public String[] getMonitoredTables() {
|
||||
return streamingMeter.getCapturedTables();
|
||||
}
|
||||
|
||||
@Override
|
||||
public String[] getCapturedTables() {
|
||||
return streamingMeter.getCapturedTables();
|
||||
|
@ -88,7 +88,7 @@ By default, the connector's snapshot mode is set to `initial`.
|
||||
When the snapshot mode is set to the default, the connector completes the following tasks to create a snapshot:
|
||||
|
||||
1. Determines the tables to be captured.
|
||||
2. Obtains a `ROW SHARE MODE` lock on each of the monitored tables to prevent structural changes from occurring during creation of the snapshot. {prodname} holds the locks for only a short time.
|
||||
2. Obtains a `ROW SHARE MODE` lock on each of the captured tables to prevent structural changes from occurring during creation of the snapshot. {prodname} holds the locks for only a short time.
|
||||
3. Reads the current system change number (SCN) position from the server's redo log.
|
||||
4. Captures the structure of all relevant tables.
|
||||
5. Releases the locks obtained in Step 2.
|
||||
@ -2573,13 +2573,13 @@ In environments that use the LogMiner implementation, you must use POSIX regular
|
||||
|
||||
|[[oracle-property-table-include-list]]<<oracle-property-table-include-list, `+table.include.list+`>>
|
||||
|No default
|
||||
|An optional comma-separated list of regular expressions that match fully-qualified table identifiers for tables to be monitored.
|
||||
|An optional comma-separated list of regular expressions that match fully-qualified table identifiers for tables to be captured.
|
||||
Tables that are not included in the include list are excluded from monitoring.
|
||||
Each table identifier uses the following format: +
|
||||
+
|
||||
`__<schema_name>.<table_name>__` +
|
||||
+
|
||||
By default, the connector monitors every non-system table in each monitored database.
|
||||
By default, the connector monitors every non-system table in each captured database.
|
||||
Do not use this property in combination with `table.exclude.list`.
|
||||
If you use the LogMiner implementation, use only POSIX regular expressions with this property.
|
||||
|
||||
|
@ -34,13 +34,6 @@ The following table lists the shapshot metrics that are available.
|
||||
|`long`
|
||||
| The number of events that have been filtered by include/exclude list filtering rules configured on the connector.
|
||||
|
||||
ifdef::product[]
|
||||
|[[connectors-snaps-metric-monitoredtables_{context}]]<<connectors-snaps-metric-monitoredtables_{context}, `MonitoredTables`>> +
|
||||
_Deprecated and scheduled for removal in a future release; use the `CapturedTables` metric instead._
|
||||
|`string[]`
|
||||
|The list of tables that are monitored by the connector.
|
||||
endif::product[]
|
||||
|
||||
|[[connectors-snaps-metric-capturedtables_{context}]]<<connectors-snaps-metric-capturedtables_{context}, `CapturedTables`>>
|
||||
|`string[]`
|
||||
|The list of tables that are captured by the connector.
|
||||
|
@ -44,13 +44,6 @@ The following table lists the streaming metrics that are available.
|
||||
|`long`
|
||||
|The number of events that have been filtered by include/exclude list filtering rules configured on the connector.
|
||||
|
||||
ifdef::product[]
|
||||
|[[connectors-strm-metric-monitoredtables_{context}]]<<connectors-strm-metric-monitoredtables_{context}, `MonitoredTables`>> +
|
||||
_Deprecated and scheduled for removal in a future release; use the 'CapturedTables' metric instead_
|
||||
|`string[]`
|
||||
|The list of tables that are monitored by the connector.
|
||||
endif::product[]
|
||||
|
||||
|[[connectors-strm-metric-capturedtables_{context}]]<<connectors-strm-metric-capturedtables_{context}, `CapturedTables`>>
|
||||
|`string[]`
|
||||
|The list of tables that are captured by the connector.
|
||||
|
Loading…
Reference in New Issue
Block a user