DBZ-6899 Refactor Oracle streamings metrics

This commit is contained in:
Chris Cranford 2023-09-25 21:01:12 -04:00 committed by Chris Cranford
parent e8e7c705af
commit a3b51b48b9
35 changed files with 2137 additions and 1627 deletions

View File

@ -0,0 +1,127 @@
/*
* Copyright Debezium Authors.
*
* Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0
*/
package io.debezium.connector.oracle;
import java.util.concurrent.atomic.AtomicLong;
import io.debezium.annotation.ThreadSafe;
import io.debezium.connector.base.ChangeEventQueueMetrics;
import io.debezium.connector.common.CdcSourceTaskContext;
import io.debezium.pipeline.metrics.DefaultStreamingChangeEventSourceMetrics;
import io.debezium.pipeline.source.spi.EventMetadataProvider;
/**
* Common Oracle Streaming Metrics for all connector adapters.
*
* @author Chris Cranford
*/
@ThreadSafe
public abstract class AbstractOracleStreamingChangeEventSourceMetrics
extends DefaultStreamingChangeEventSourceMetrics<OraclePartition>
implements OracleCommonStreamingChangeEventSourceMetricsMXBean {
private final AtomicLong schemaChangeParseErrorCount = new AtomicLong();
private final AtomicLong committedTransactionCount = new AtomicLong();
private final AtomicLong lastCapturedDmlCount = new AtomicLong();
private final AtomicLong maxCapturedDmlCount = new AtomicLong();
private final AtomicLong totalCapturedDmlCount = new AtomicLong();
private final AtomicLong warningCount = new AtomicLong();
private final AtomicLong errorCount = new AtomicLong();
public AbstractOracleStreamingChangeEventSourceMetrics(CdcSourceTaskContext taskContext,
ChangeEventQueueMetrics changeEventQueueMetrics,
EventMetadataProvider metadataProvider) {
super(taskContext, changeEventQueueMetrics, metadataProvider);
}
@Override
public void reset() {
super.reset();
committedTransactionCount.set(0);
lastCapturedDmlCount.set(0);
maxCapturedDmlCount.set(0);
totalCapturedDmlCount.set(0);
warningCount.set(0);
errorCount.set(0);
}
@Override
public long getNumberOfCommittedTransactions() {
return committedTransactionCount.get();
}
@Override
public long getTotalSchemaChangeParseErrorCount() {
return schemaChangeParseErrorCount.get();
}
@Override
public long getLastCapturedDmlCount() {
return lastCapturedDmlCount.get();
}
@Override
public long getMaxCapturedDmlCountInBatch() {
return maxCapturedDmlCount.get();
}
@Override
public long getTotalCapturedDmlCount() {
return totalCapturedDmlCount.get();
}
@Override
public long getWarningCount() {
return warningCount.get();
}
@Override
public long getErrorCount() {
return errorCount.get();
}
/**
* Set the last iteration's number of data manipulation (insert, update, delete) events.
*
* @param lastDmlCount the last number of insert, update, and delete events
*/
public void setLastCapturedDmlCount(int lastDmlCount) {
lastCapturedDmlCount.set(lastDmlCount);
if (maxCapturedDmlCount.get() < lastDmlCount) {
maxCapturedDmlCount.set(lastDmlCount);
}
totalCapturedDmlCount.getAndAdd(lastDmlCount);
}
/**
* Increments the total number of schema change parser errors.
*/
public void incrementSchemaChangeParseErrorCount() {
schemaChangeParseErrorCount.incrementAndGet();
}
/**
* Increments the number of warning messages written to the connector log.
*/
public void incrementWarningCount() {
warningCount.incrementAndGet();
}
/**
* Increments the number of error messages written to the connector log.
*/
public void incrementErrorCount() {
errorCount.incrementAndGet();
}
/**
* Increments the committed transaction count.
*/
public void incrementCommittedTransactionCount() {
committedTransactionCount.incrementAndGet();
}
}

View File

@ -22,7 +22,7 @@
*
* @author Chris Cranford
*/
public abstract class AbstractStreamingAdapter implements StreamingAdapter {
public abstract class AbstractStreamingAdapter<T extends AbstractOracleStreamingChangeEventSourceMetrics> implements StreamingAdapter<T> {
private static final Logger LOGGER = LoggerFactory.getLogger(AbstractStreamingAdapter.class);

View File

@ -33,12 +33,12 @@ public class OracleChangeEventSourceFactory implements ChangeEventSourceFactory<
private final OracleDatabaseSchema schema;
private final Configuration jdbcConfig;
private final OracleTaskContext taskContext;
private final OracleStreamingChangeEventSourceMetrics streamingMetrics;
private final AbstractOracleStreamingChangeEventSourceMetrics streamingMetrics;
public OracleChangeEventSourceFactory(OracleConnectorConfig configuration, MainConnectionProvidingConnectionFactory<OracleConnection> connectionFactory,
ErrorHandler errorHandler, EventDispatcher<OraclePartition, TableId> dispatcher, Clock clock, OracleDatabaseSchema schema,
Configuration jdbcConfig, OracleTaskContext taskContext,
OracleStreamingChangeEventSourceMetrics streamingMetrics) {
AbstractOracleStreamingChangeEventSourceMetrics streamingMetrics) {
this.configuration = configuration;
this.connectionFactory = connectionFactory;
this.errorHandler = errorHandler;

View File

@ -16,9 +16,9 @@
*/
public class OracleChangeEventSourceMetricsFactory extends DefaultChangeEventSourceMetricsFactory<OraclePartition> {
private final OracleStreamingChangeEventSourceMetrics streamingMetrics;
private final AbstractOracleStreamingChangeEventSourceMetrics streamingMetrics;
public OracleChangeEventSourceMetricsFactory(OracleStreamingChangeEventSourceMetrics streamingMetrics) {
public OracleChangeEventSourceMetricsFactory(AbstractOracleStreamingChangeEventSourceMetrics streamingMetrics) {
this.streamingMetrics = streamingMetrics;
}

View File

@ -0,0 +1,67 @@
/*
* Copyright Debezium Authors.
*
* Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0
*/
package io.debezium.connector.oracle;
import io.debezium.pipeline.metrics.StreamingChangeEventSourceMetricsMXBean;
/**
* Common metrics across all Oracle streaming adapters.
*
* @author Chris Cranford
*/
public interface OracleCommonStreamingChangeEventSourceMetricsMXBean extends StreamingChangeEventSourceMetricsMXBean {
@Deprecated
default long getMaxCapturedDmlInBatch() {
return getMaxCapturedDmlCountInBatch();
}
@Deprecated
default long getNetworkConnectionProblemsCounter() {
// Was used specifically by Oracle tests previously and not in runtime code.
return 0L;
}
/**
* @return total number of schema change parser errors
* @deprecated to be removed in Debezium 2.7, replaced by {{@link #getTotalSchemaChangeParseErrorCount()}}
*/
@Deprecated
default long getUnparsableDdlCount() {
return getTotalSchemaChangeParseErrorCount();
}
/**
* @return total number of schema change events that resulted in a parser failure
*/
long getTotalSchemaChangeParseErrorCount();
/**
* @return the number of data manipulation (insert, update, delete) events during the last batch
*/
long getLastCapturedDmlCount();
/**
* @return maximum number of data manipulation (insert, update, delete) events during a single batch
*/
long getMaxCapturedDmlCountInBatch();
/**
* @return the number of captured data manipulation (insert, update, delete) events
*/
long getTotalCapturedDmlCount();
/**
* @return number of warnings detected by the connector
*/
long getWarningCount();
/**
* @return number of errors detected by the connector
*/
long getErrorCount();
}

View File

@ -33,6 +33,7 @@
import io.debezium.config.Instantiator;
import io.debezium.connector.AbstractSourceInfo;
import io.debezium.connector.SourceInfoStructMaker;
import io.debezium.connector.oracle.logminer.LogMinerStreamingChangeEventSourceMetrics;
import io.debezium.connector.oracle.logminer.logwriter.LogWriterFlushStrategy;
import io.debezium.connector.oracle.logminer.processor.LogMinerEventProcessor;
import io.debezium.connector.oracle.logminer.processor.infinispan.EmbeddedInfinispanLogMinerEventProcessor;
@ -1307,7 +1308,7 @@ public LogMinerEventProcessor createProcessor(ChangeEventSourceContext context,
OraclePartition partition,
OracleOffsetContext offsetContext,
OracleDatabaseSchema schema,
OracleStreamingChangeEventSourceMetrics metrics) {
LogMinerStreamingChangeEventSourceMetrics metrics) {
return new MemoryLogMinerEventProcessor(context, connectorConfig, connection, dispatcher, partition,
offsetContext, schema, metrics);
}
@ -1322,7 +1323,7 @@ public LogMinerEventProcessor createProcessor(ChangeEventSourceContext context,
OraclePartition partition,
OracleOffsetContext offsetContext,
OracleDatabaseSchema schema,
OracleStreamingChangeEventSourceMetrics metrics) {
LogMinerStreamingChangeEventSourceMetrics metrics) {
return new EmbeddedInfinispanLogMinerEventProcessor(context, connectorConfig, connection, dispatcher,
partition, offsetContext, schema, metrics);
}
@ -1337,7 +1338,7 @@ public LogMinerEventProcessor createProcessor(ChangeEventSourceContext context,
OraclePartition partition,
OracleOffsetContext offsetContext,
OracleDatabaseSchema schema,
OracleStreamingChangeEventSourceMetrics metrics) {
LogMinerStreamingChangeEventSourceMetrics metrics) {
return new RemoteInfinispanLogMinerEventProcessor(context, connectorConfig, connection, dispatcher,
partition, offsetContext, schema, metrics);
}
@ -1352,7 +1353,7 @@ public abstract LogMinerEventProcessor createProcessor(ChangeEventSourceContext
OracleConnection connection, EventDispatcher<OraclePartition, TableId> dispatcher,
OraclePartition partition,
OracleOffsetContext offsetContext, OracleDatabaseSchema schema,
OracleStreamingChangeEventSourceMetrics metrics);
LogMinerStreamingChangeEventSourceMetrics metrics);
LogMiningBufferType(String value) {
this.value = value;

View File

@ -124,8 +124,8 @@ public ChangeEventSourceCoordinator<OraclePartition, OracleOffsetContext> start(
schemaNameAdjuster,
signalProcessor);
final OracleStreamingChangeEventSourceMetrics streamingMetrics = new OracleStreamingChangeEventSourceMetrics(taskContext, queue, metadataProvider,
connectorConfig);
final AbstractOracleStreamingChangeEventSourceMetrics streamingMetrics = connectorConfig.getAdapter()
.getStreamingMetrics(taskContext, queue, metadataProvider, connectorConfig);
NotificationService<OraclePartition, OracleOffsetContext> notificationService = new NotificationService<>(getNotificationChannels(),
connectorConfig, SchemaFactory.get(), dispatcher::enqueueNotification);

View File

@ -47,13 +47,13 @@ public class OracleSchemaChangeEventEmitter implements SchemaChangeEventEmitter
private final String objectOwner;
private final String ddlText;
private final TableFilter filters;
private final OracleStreamingChangeEventSourceMetrics streamingMetrics;
private final AbstractOracleStreamingChangeEventSourceMetrics streamingMetrics;
private final TruncateReceiver truncateReceiver;
public OracleSchemaChangeEventEmitter(OracleConnectorConfig connectorConfig, OraclePartition partition,
OracleOffsetContext offsetContext, TableId tableId, String sourceDatabaseName,
String objectOwner, String ddlText, OracleDatabaseSchema schema,
Instant changeTime, OracleStreamingChangeEventSourceMetrics streamingMetrics,
Instant changeTime, AbstractOracleStreamingChangeEventSourceMetrics streamingMetrics,
TruncateReceiver truncateReceiver) {
this.partition = partition;
this.offsetContext = offsetContext;
@ -85,9 +85,9 @@ public void emitSchemaChangeEvent(Receiver receiver) throws InterruptedException
}
catch (ParsingException | MultipleParsingExceptions e) {
if (schema.skipUnparseableDdlStatements()) {
LOGGER.warn("Ignoring unparsable DDL statement '{}': {}", ddlText, e);
LOGGER.warn("Ignoring unparsable DDL statement '{}':", ddlText, e);
streamingMetrics.incrementWarningCount();
streamingMetrics.incrementUnparsableDdlCount();
streamingMetrics.incrementSchemaChangeParseErrorCount();
}
else {
throw e;

View File

@ -1,838 +0,0 @@
/*
* Copyright Debezium Authors.
*
* Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0
*/
package io.debezium.connector.oracle;
import java.math.BigInteger;
import java.time.Clock;
import java.time.Duration;
import java.time.Instant;
import java.time.OffsetDateTime;
import java.time.ZoneOffset;
import java.util.Arrays;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicLong;
import java.util.concurrent.atomic.AtomicReference;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import io.debezium.annotation.ThreadSafe;
import io.debezium.annotation.VisibleForTesting;
import io.debezium.connector.base.ChangeEventQueueMetrics;
import io.debezium.connector.common.CdcSourceTaskContext;
import io.debezium.pipeline.metrics.DefaultStreamingChangeEventSourceMetrics;
import io.debezium.pipeline.source.spi.EventMetadataProvider;
import io.debezium.util.LRUCacheMap;
/**
* The metrics implementation for Oracle connector streaming phase.
*/
@ThreadSafe
public class OracleStreamingChangeEventSourceMetrics extends DefaultStreamingChangeEventSourceMetrics<OraclePartition>
implements OracleStreamingChangeEventSourceMetricsMXBean {
private static final Logger LOGGER = LoggerFactory.getLogger(OracleStreamingChangeEventSourceMetrics.class);
private static final long MILLIS_PER_SECOND = 1000L;
private static final int TRANSACTION_ID_SET_SIZE = 10;
private final AtomicReference<Scn> currentScn = new AtomicReference<>();
private final AtomicInteger logMinerQueryCount = new AtomicInteger();
private final AtomicInteger totalCapturedDmlCount = new AtomicInteger();
private final AtomicReference<Duration> totalDurationOfFetchingQuery = new AtomicReference<>();
private final AtomicInteger lastCapturedDmlCount = new AtomicInteger();
private final AtomicReference<Duration> lastDurationOfFetchingQuery = new AtomicReference<>();
private final AtomicLong maxCapturedDmlCount = new AtomicLong();
private final AtomicLong totalProcessedRows = new AtomicLong();
private final AtomicReference<Duration> maxDurationOfFetchingQuery = new AtomicReference<>();
private final AtomicReference<Duration> totalBatchProcessingDuration = new AtomicReference<>();
private final AtomicReference<Duration> lastBatchProcessingDuration = new AtomicReference<>();
private final AtomicReference<Duration> totalParseTime = new AtomicReference<>();
private final AtomicReference<Duration> totalStartLogMiningSessionDuration = new AtomicReference<>();
private final AtomicReference<Duration> lastStartLogMiningSessionDuration = new AtomicReference<>();
private final AtomicReference<Duration> maxStartingLogMiningSessionDuration = new AtomicReference<>();
private final AtomicReference<Duration> totalProcessingTime = new AtomicReference<>();
private final AtomicReference<Duration> minBatchProcessingTime = new AtomicReference<>();
private final AtomicReference<Duration> maxBatchProcessingTime = new AtomicReference<>();
private final AtomicReference<Duration> totalResultSetNextTime = new AtomicReference<>();
private final AtomicLong maxBatchProcessingThroughput = new AtomicLong();
private final AtomicReference<String[]> currentLogFileName;
private final AtomicReference<String[]> redoLogStatus;
private final AtomicLong minimumLogsMined = new AtomicLong();
private final AtomicLong maximumLogsMined = new AtomicLong();
private final AtomicInteger switchCounter = new AtomicInteger();
private final AtomicInteger batchSize = new AtomicInteger();
private final AtomicLong millisecondToSleepBetweenMiningQuery = new AtomicLong();
private final AtomicLong networkConnectionProblemsCounter = new AtomicLong();
private final AtomicReference<Duration> keepTransactionsDuration = new AtomicReference<>();
private final AtomicReference<Duration> lagFromTheSourceDuration = new AtomicReference<>();
private final AtomicReference<Duration> minLagFromTheSourceDuration = new AtomicReference<>();
private final AtomicReference<Duration> maxLagFromTheSourceDuration = new AtomicReference<>();
private final AtomicReference<Duration> lastCommitDuration = new AtomicReference<>();
private final AtomicReference<Duration> maxCommitDuration = new AtomicReference<>();
private final AtomicLong activeTransactions = new AtomicLong();
private final AtomicLong rolledBackTransactions = new AtomicLong();
private final AtomicLong committedTransactions = new AtomicLong();
private final AtomicLong oversizedTransactions = new AtomicLong();
private final AtomicReference<LRUCacheMap<String, String>> abandonedTransactionIds = new AtomicReference<>();
private final AtomicReference<LRUCacheMap<String, String>> rolledBackTransactionIds = new AtomicReference<>();
private final AtomicLong registeredDmlCount = new AtomicLong();
private final AtomicLong committedDmlCount = new AtomicLong();
private final AtomicInteger errorCount = new AtomicInteger();
private final AtomicInteger warningCount = new AtomicInteger();
private final AtomicInteger scnFreezeCount = new AtomicInteger();
private final AtomicLong timeDifference = new AtomicLong();
private final AtomicReference<ZoneOffset> zoneOffset = new AtomicReference<>();
private final AtomicReference<Instant> oldestScnAge = new AtomicReference<>();
private final AtomicReference<Scn> oldestScn = new AtomicReference<>();
private final AtomicReference<Scn> committedScn = new AtomicReference<>();
private final AtomicReference<Scn> offsetScn = new AtomicReference<>();
private final AtomicInteger unparsableDdlCount = new AtomicInteger();
private final AtomicLong miningSessionUserGlobalAreaMemory = new AtomicLong();
private final AtomicLong miningSessionUserGlobalAreaMaxMemory = new AtomicLong();
private final AtomicLong miningSessionProcessGlobalAreaMemory = new AtomicLong();
private final AtomicLong miningSessionProcessGlobalAreaMaxMemory = new AtomicLong();
// Constants for sliding window algorithm
private final int batchSizeMin;
private final int batchSizeMax;
private final int batchSizeDefault;
// constants for sleeping algorithm
private final long sleepTimeMin;
private final long sleepTimeMax;
private final long sleepTimeDefault;
private final long sleepTimeIncrement;
private final Instant startTime;
private final Clock clock;
public OracleStreamingChangeEventSourceMetrics(CdcSourceTaskContext taskContext, ChangeEventQueueMetrics changeEventQueueMetrics,
EventMetadataProvider metadataProvider,
OracleConnectorConfig connectorConfig) {
this(taskContext, changeEventQueueMetrics, metadataProvider, connectorConfig, Clock.systemUTC());
}
/**
* Constructor that allows providing a clock to be used for Tests.
*/
@VisibleForTesting
OracleStreamingChangeEventSourceMetrics(CdcSourceTaskContext taskContext, ChangeEventQueueMetrics changeEventQueueMetrics,
EventMetadataProvider metadataProvider,
OracleConnectorConfig connectorConfig,
Clock clock) {
super(taskContext, changeEventQueueMetrics, metadataProvider);
this.clock = clock;
startTime = clock.instant();
timeDifference.set(0L);
zoneOffset.set(ZoneOffset.UTC);
currentScn.set(Scn.NULL);
oldestScn.set(Scn.NULL);
offsetScn.set(Scn.NULL);
committedScn.set(Scn.NULL);
currentLogFileName = new AtomicReference<>(new String[0]);
minimumLogsMined.set(0L);
maximumLogsMined.set(0L);
redoLogStatus = new AtomicReference<>(new String[0]);
switchCounter.set(0);
batchSizeDefault = connectorConfig.getLogMiningBatchSizeDefault();
batchSizeMin = connectorConfig.getLogMiningBatchSizeMin();
batchSizeMax = connectorConfig.getLogMiningBatchSizeMax();
sleepTimeDefault = connectorConfig.getLogMiningSleepTimeDefault().toMillis();
sleepTimeMin = connectorConfig.getLogMiningSleepTimeMin().toMillis();
sleepTimeMax = connectorConfig.getLogMiningSleepTimeMax().toMillis();
sleepTimeIncrement = connectorConfig.getLogMiningSleepTimeIncrement().toMillis();
keepTransactionsDuration.set(connectorConfig.getLogMiningTransactionRetention());
reset();
}
@Override
public void reset() {
batchSize.set(batchSizeDefault);
millisecondToSleepBetweenMiningQuery.set(sleepTimeDefault);
totalCapturedDmlCount.set(0);
totalProcessedRows.set(0);
maxDurationOfFetchingQuery.set(Duration.ZERO);
lastDurationOfFetchingQuery.set(Duration.ZERO);
logMinerQueryCount.set(0);
totalDurationOfFetchingQuery.set(Duration.ZERO);
lastCapturedDmlCount.set(0);
maxCapturedDmlCount.set(0);
totalBatchProcessingDuration.set(Duration.ZERO);
maxBatchProcessingThroughput.set(0);
lastBatchProcessingDuration.set(Duration.ZERO);
networkConnectionProblemsCounter.set(0);
totalParseTime.set(Duration.ZERO);
totalStartLogMiningSessionDuration.set(Duration.ZERO);
lastStartLogMiningSessionDuration.set(Duration.ZERO);
maxStartingLogMiningSessionDuration.set(Duration.ZERO);
totalProcessingTime.set(Duration.ZERO);
minBatchProcessingTime.set(Duration.ZERO);
maxBatchProcessingTime.set(Duration.ZERO);
totalResultSetNextTime.set(Duration.ZERO);
miningSessionUserGlobalAreaMemory.set(0L);
miningSessionUserGlobalAreaMaxMemory.set(0L);
miningSessionProcessGlobalAreaMemory.set(0L);
miningSessionProcessGlobalAreaMaxMemory.set(0L);
// transactional buffer metrics
lagFromTheSourceDuration.set(Duration.ZERO);
maxLagFromTheSourceDuration.set(Duration.ZERO);
minLagFromTheSourceDuration.set(Duration.ZERO);
lastCommitDuration.set(Duration.ZERO);
maxCommitDuration.set(Duration.ZERO);
activeTransactions.set(0);
rolledBackTransactions.set(0);
committedTransactions.set(0);
oversizedTransactions.set(0);
registeredDmlCount.set(0);
committedDmlCount.set(0);
abandonedTransactionIds.set(new LRUCacheMap<>(TRANSACTION_ID_SET_SIZE));
rolledBackTransactionIds.set(new LRUCacheMap<>(TRANSACTION_ID_SET_SIZE));
errorCount.set(0);
warningCount.set(0);
scnFreezeCount.set(0);
oldestScnAge.set(null);
}
public void setCurrentScn(Scn scn) {
currentScn.set(scn);
}
public void setCurrentLogFileName(Set<String> names) {
currentLogFileName.set(names.stream().toArray(String[]::new));
if (names.size() < minimumLogsMined.get()) {
minimumLogsMined.set(names.size());
}
else if (minimumLogsMined.get() == 0) {
minimumLogsMined.set(names.size());
}
if (names.size() > maximumLogsMined.get()) {
maximumLogsMined.set(names.size());
}
}
@Override
public long getMinimumMinedLogCount() {
return minimumLogsMined.get();
}
@Override
public long getMaximumMinedLogCount() {
return maximumLogsMined.get();
}
public void setRedoLogStatus(Map<String, String> status) {
String[] statusArray = status.entrySet().stream().map(e -> e.getKey() + " | " + e.getValue()).toArray(String[]::new);
redoLogStatus.set(statusArray);
}
public void setSwitchCount(int counter) {
switchCounter.set(counter);
}
public void setLastCapturedDmlCount(int dmlCount) {
lastCapturedDmlCount.set(dmlCount);
if (dmlCount > maxCapturedDmlCount.get()) {
maxCapturedDmlCount.set(dmlCount);
}
totalCapturedDmlCount.getAndAdd(dmlCount);
}
public void setLastDurationOfBatchCapturing(Duration lastDuration) {
lastDurationOfFetchingQuery.set(lastDuration);
totalDurationOfFetchingQuery.accumulateAndGet(lastDurationOfFetchingQuery.get(), Duration::plus);
if (maxDurationOfFetchingQuery.get().toMillis() < lastDurationOfFetchingQuery.get().toMillis()) {
maxDurationOfFetchingQuery.set(lastDuration);
}
logMinerQueryCount.incrementAndGet();
}
public void setLastDurationOfBatchProcessing(Duration lastDuration) {
lastBatchProcessingDuration.set(lastDuration);
totalBatchProcessingDuration.accumulateAndGet(lastDuration, Duration::plus);
if (maxBatchProcessingTime.get().toMillis() < lastDuration.toMillis()) {
maxBatchProcessingTime.set(lastDuration);
}
if (minBatchProcessingTime.get().toMillis() > lastDuration.toMillis()) {
minBatchProcessingTime.set(lastDuration);
}
else if (minBatchProcessingTime.get().toMillis() == 0L) {
minBatchProcessingTime.set(lastDuration);
}
if (getLastBatchProcessingThroughput() > maxBatchProcessingThroughput.get()) {
maxBatchProcessingThroughput.set(getLastBatchProcessingThroughput());
}
}
public void incrementNetworkConnectionProblemsCounter() {
networkConnectionProblemsCounter.incrementAndGet();
}
@Override
public BigInteger getCurrentScn() {
return currentScn.get().asBigInteger();
}
@Override
public long getTotalCapturedDmlCount() {
return totalCapturedDmlCount.get();
}
@Override
public String[] getCurrentRedoLogFileName() {
return currentLogFileName.get();
}
@Override
public String[] getRedoLogStatus() {
return redoLogStatus.get();
}
@Override
public int getSwitchCounter() {
return switchCounter.get();
}
@Override
public Long getLastDurationOfFetchQueryInMilliseconds() {
return lastDurationOfFetchingQuery.get() == null ? 0 : lastDurationOfFetchingQuery.get().toMillis();
}
@Override
public long getLastBatchProcessingTimeInMilliseconds() {
return lastBatchProcessingDuration.get().toMillis();
}
@Override
public Long getMaxDurationOfFetchQueryInMilliseconds() {
return maxDurationOfFetchingQuery.get() == null ? 0 : maxDurationOfFetchingQuery.get().toMillis();
}
@Override
public Long getMaxCapturedDmlInBatch() {
return maxCapturedDmlCount.get();
}
@Override
public int getLastCapturedDmlCount() {
return lastCapturedDmlCount.get();
}
@Override
public long getTotalProcessedRows() {
return totalProcessedRows.get();
}
@Override
public long getTotalResultSetNextTimeInMilliseconds() {
return totalResultSetNextTime.get().toMillis();
}
@Override
public long getAverageBatchProcessingThroughput() {
if (totalBatchProcessingDuration.get().isZero()) {
return 0L;
}
return Math.round((totalCapturedDmlCount.floatValue() / totalBatchProcessingDuration.get().toMillis()) * 1000);
}
@Override
public long getLastBatchProcessingThroughput() {
if (lastBatchProcessingDuration.get().isZero()) {
return 0L;
}
return Math.round((lastCapturedDmlCount.floatValue() / lastBatchProcessingDuration.get().toMillis()) * 1000);
}
@Override
public long getFetchingQueryCount() {
return logMinerQueryCount.get();
}
@Override
public int getBatchSize() {
return batchSize.get();
}
@Override
public long getMillisecondToSleepBetweenMiningQuery() {
return millisecondToSleepBetweenMiningQuery.get();
}
@Override
public int getHoursToKeepTransactionInBuffer() {
return (int) keepTransactionsDuration.get().toHours();
}
@Override
public long getMillisecondsToKeepTransactionsInBuffer() {
return keepTransactionsDuration.get().toMillis();
}
@Override
public long getMaxBatchProcessingThroughput() {
return maxBatchProcessingThroughput.get();
}
@Override
public long getNetworkConnectionProblemsCounter() {
return networkConnectionProblemsCounter.get();
}
@Override
public long getTotalParseTimeInMilliseconds() {
return totalParseTime.get().toMillis();
}
public void addCurrentParseTime(Duration currentParseTime) {
totalParseTime.accumulateAndGet(currentParseTime, Duration::plus);
}
@Override
public long getTotalMiningSessionStartTimeInMilliseconds() {
return totalStartLogMiningSessionDuration.get().toMillis();
}
public void addCurrentMiningSessionStart(Duration currentStartLogMiningSession) {
lastStartLogMiningSessionDuration.set(currentStartLogMiningSession);
if (currentStartLogMiningSession.compareTo(maxStartingLogMiningSessionDuration.get()) > 0) {
maxStartingLogMiningSessionDuration.set(currentStartLogMiningSession);
}
totalStartLogMiningSessionDuration.accumulateAndGet(currentStartLogMiningSession, Duration::plus);
}
@Override
public long getLastMiningSessionStartTimeInMilliseconds() {
return lastStartLogMiningSessionDuration.get().toMillis();
}
@Override
public long getMaxMiningSessionStartTimeInMilliseconds() {
return maxStartingLogMiningSessionDuration.get().toMillis();
}
@Override
public long getTotalProcessingTimeInMilliseconds() {
return totalProcessingTime.get().toMillis();
}
@Override
public long getMinBatchProcessingTimeInMilliseconds() {
return minBatchProcessingTime.get().toMillis();
}
@Override
public long getMaxBatchProcessingTimeInMilliseconds() {
return maxBatchProcessingTime.get().toMillis();
}
public void setCurrentBatchProcessingTime(Duration currentBatchProcessingTime) {
totalProcessingTime.accumulateAndGet(currentBatchProcessingTime, Duration::plus);
setLastDurationOfBatchProcessing(currentBatchProcessingTime);
}
public void addCurrentResultSetNext(Duration currentNextTime) {
totalResultSetNextTime.accumulateAndGet(currentNextTime, Duration::plus);
}
public void addProcessedRows(Long rows) {
totalProcessedRows.getAndAdd(rows);
}
@Override
public void setBatchSize(int size) {
if (size >= batchSizeMin && size <= batchSizeMax) {
batchSize.set(size);
}
}
@Override
public void setMillisecondToSleepBetweenMiningQuery(long milliseconds) {
if (milliseconds >= sleepTimeMin && milliseconds < sleepTimeMax) {
millisecondToSleepBetweenMiningQuery.set(milliseconds);
}
}
@Override
public void changeSleepingTime(boolean increment) {
long sleepTime = millisecondToSleepBetweenMiningQuery.get();
if (increment && sleepTime < sleepTimeMax) {
sleepTime = millisecondToSleepBetweenMiningQuery.addAndGet(sleepTimeIncrement);
}
else if (sleepTime > sleepTimeMin) {
sleepTime = millisecondToSleepBetweenMiningQuery.addAndGet(-sleepTimeIncrement);
}
LOGGER.debug("Updating sleep time window. Sleep time {}. Min sleep time {}. Max sleep time {}.", sleepTime, sleepTimeMin, sleepTimeMax);
}
@Override
public void changeBatchSize(boolean increment, boolean lobEnabled) {
int currentBatchSize = batchSize.get();
boolean incremented = false;
if (increment && currentBatchSize < batchSizeMax) {
currentBatchSize = batchSize.addAndGet(batchSizeMin);
incremented = true;
}
else if (!increment && currentBatchSize > batchSizeMin) {
currentBatchSize = batchSize.addAndGet(-batchSizeMin);
}
if (incremented && currentBatchSize == batchSizeMax) {
if (!lobEnabled) {
LOGGER.info("The connector is now using the maximum batch size {} when querying the LogMiner view. This could be indicative of large SCN gaps",
currentBatchSize);
}
else {
LOGGER.info("The connector is now using the maximum batch size {} when querying the LogMiner view.", currentBatchSize);
}
}
else {
LOGGER.debug("Updating batch size window. Batch size {}. Min batch size {}. Max batch size {}.", currentBatchSize, batchSizeMin, batchSizeMax);
}
}
// transactional buffer metrics
@Override
public long getNumberOfActiveTransactions() {
return activeTransactions.get();
}
@Override
public long getNumberOfRolledBackTransactions() {
return rolledBackTransactions.get();
}
@Override
public long getNumberOfCommittedTransactions() {
return committedTransactions.get();
}
@Override
public long getNumberOfOversizedTransactions() {
return oversizedTransactions.get();
}
@Override
public long getCommitThroughput() {
long timeSpent = Duration.between(startTime, clock.instant()).toMillis();
return committedTransactions.get() * MILLIS_PER_SECOND / (timeSpent != 0 ? timeSpent : 1);
}
@Override
public long getRegisteredDmlCount() {
return registeredDmlCount.get();
}
@Override
public long getOldestScnAgeInMilliseconds() {
if (oldestScnAge.get() == null) {
return 0L;
}
return Duration.between(Instant.now(), oldestScnAge.get()).toMillis();
}
@Override
public BigInteger getOldestScn() {
return oldestScn.get().asBigInteger();
}
@Override
public BigInteger getCommittedScn() {
return committedScn.get().asBigInteger();
}
@Override
public BigInteger getOffsetScn() {
return offsetScn.get().asBigInteger();
}
@Override
public long getLagFromSourceInMilliseconds() {
return lagFromTheSourceDuration.get().toMillis();
}
@Override
public long getMaxLagFromSourceInMilliseconds() {
return maxLagFromTheSourceDuration.get().toMillis();
}
@Override
public long getMinLagFromSourceInMilliseconds() {
return minLagFromTheSourceDuration.get().toMillis();
}
@Override
public Set<String> getAbandonedTransactionIds() {
return abandonedTransactionIds.get().keySet();
}
@Override
public Set<String> getRolledBackTransactionIds() {
return rolledBackTransactionIds.get().keySet();
}
@Override
public long getLastCommitDurationInMilliseconds() {
return lastCommitDuration.get().toMillis();
}
@Override
public long getMaxCommitDurationInMilliseconds() {
return maxCommitDuration.get().toMillis();
}
@Override
public int getErrorCount() {
return errorCount.get();
}
@Override
public int getWarningCount() {
return warningCount.get();
}
@Override
public int getScnFreezeCount() {
return scnFreezeCount.get();
}
@Override
public int getUnparsableDdlCount() {
return unparsableDdlCount.get();
}
@Override
public long getMiningSessionUserGlobalAreaMemoryInBytes() {
return miningSessionUserGlobalAreaMemory.get();
}
@Override
public long getMiningSessionUserGlobalAreaMaxMemoryInBytes() {
return miningSessionUserGlobalAreaMaxMemory.get();
}
@Override
public long getMiningSessionProcessGlobalAreaMemoryInBytes() {
return miningSessionProcessGlobalAreaMemory.get();
}
@Override
public long getMiningSessionProcessGlobalAreaMaxMemoryInBytes() {
return miningSessionProcessGlobalAreaMaxMemory.get();
}
public void setOldestScnAge(Instant changeTime) {
oldestScnAge.set(changeTime);
}
public void setOldestScn(Scn scn) {
oldestScn.set(scn);
}
public void setCommittedScn(Scn scn) {
committedScn.set(scn);
}
public void setOffsetScn(Scn scn) {
offsetScn.set(scn);
}
public void setActiveTransactions(long activeTransactionCount) {
activeTransactions.set(activeTransactionCount);
}
public void incrementRolledBackTransactions() {
rolledBackTransactions.incrementAndGet();
}
public void incrementCommittedTransactions() {
committedTransactions.incrementAndGet();
}
public void incrementOversizedTransactions() {
oversizedTransactions.incrementAndGet();
}
public void incrementRegisteredDmlCount() {
registeredDmlCount.incrementAndGet();
}
public void incrementCommittedDmlCount(long counter) {
committedDmlCount.getAndAdd(counter);
}
public void incrementErrorCount() {
errorCount.incrementAndGet();
}
public void incrementWarningCount() {
warningCount.incrementAndGet();
}
public void incrementScnFreezeCount() {
scnFreezeCount.incrementAndGet();
}
public void addAbandonedTransactionId(String transactionId) {
if (transactionId != null) {
abandonedTransactionIds.get().put(transactionId, transactionId);
}
}
public void addRolledBackTransactionId(String transactionId) {
if (transactionId != null) {
rolledBackTransactionIds.get().put(transactionId, transactionId);
}
}
public void setLastCommitDuration(Duration lastDuration) {
lastCommitDuration.set(lastDuration);
if (lastDuration.toMillis() > maxCommitDuration.get().toMillis()) {
maxCommitDuration.set(lastDuration);
}
}
/**
* Calculates the time difference between the database server and the connector.
* Along with the time difference also the offset of the database server time to UTC is stored.
* Both values are required to calculate lag metrics.
*
* @param databaseSystemTime the system time (<code>SYSTIMESTAMP</code>) of the database
*/
public void calculateTimeDifference(OffsetDateTime databaseSystemTime) {
this.zoneOffset.set(databaseSystemTime.getOffset());
LOGGER.trace("Timezone offset of database system time is {} seconds", zoneOffset.get().getTotalSeconds());
Instant now = clock.instant();
long timeDiffMillis = Duration.between(databaseSystemTime.toInstant(), now).toMillis();
this.timeDifference.set(timeDiffMillis);
LOGGER.trace("Current time {} ms, database difference {} ms", now.toEpochMilli(), timeDiffMillis);
}
public ZoneOffset getDatabaseOffset() {
return zoneOffset.get();
}
public void calculateLagMetrics(Instant changeTime) {
if (changeTime != null) {
final Instant correctedChangeTime = changeTime.plusMillis(timeDifference.longValue()).minusSeconds(zoneOffset.get().getTotalSeconds());
final Duration lag = Duration.between(correctedChangeTime, clock.instant()).abs();
lagFromTheSourceDuration.set(lag);
if (maxLagFromTheSourceDuration.get().toMillis() < lag.toMillis()) {
maxLagFromTheSourceDuration.set(lag);
}
if (minLagFromTheSourceDuration.get().toMillis() > lag.toMillis()) {
minLagFromTheSourceDuration.set(lag);
}
else if (minLagFromTheSourceDuration.get().toMillis() == 0) {
minLagFromTheSourceDuration.set(lag);
}
}
}
public void incrementUnparsableDdlCount() {
unparsableDdlCount.incrementAndGet();
}
public void setUserGlobalAreaMemory(long ugaMemory, long ugaMaxMemory) {
miningSessionUserGlobalAreaMemory.set(ugaMemory);
if (ugaMaxMemory > miningSessionUserGlobalAreaMaxMemory.get()) {
miningSessionUserGlobalAreaMaxMemory.set(ugaMaxMemory);
}
}
public void setProcessGlobalAreaMemory(long pgaMemory, long pgaMaxMemory) {
miningSessionProcessGlobalAreaMemory.set(pgaMemory);
if (pgaMemory > miningSessionProcessGlobalAreaMaxMemory.get()) {
miningSessionProcessGlobalAreaMaxMemory.set(pgaMemory);
}
}
@Override
public String toString() {
return "OracleStreamingChangeEventSourceMetrics{" +
"currentScn=" + currentScn +
", oldestScn=" + oldestScn.get() +
", committedScn=" + committedScn.get() +
", offsetScn=" + offsetScn.get() +
", oldestScnChangeTime=" + oldestScnAge.get() +
", logMinerQueryCount=" + logMinerQueryCount +
", totalProcessedRows=" + totalProcessedRows +
", totalCapturedDmlCount=" + totalCapturedDmlCount +
", totalDurationOfFetchingQuery=" + totalDurationOfFetchingQuery +
", lastCapturedDmlCount=" + lastCapturedDmlCount +
", lastDurationOfFetchingQuery=" + lastDurationOfFetchingQuery +
", maxCapturedDmlCount=" + maxCapturedDmlCount +
", maxDurationOfFetchingQuery=" + maxDurationOfFetchingQuery +
", totalBatchProcessingDuration=" + totalBatchProcessingDuration +
", lastBatchProcessingDuration=" + lastBatchProcessingDuration +
", maxBatchProcessingThroughput=" + maxBatchProcessingThroughput +
", currentLogFileName=" + Arrays.asList(currentLogFileName.get()) +
", minLogFilesMined=" + minimumLogsMined +
", maxLogFilesMined=" + maximumLogsMined +
", redoLogStatus=" + Arrays.asList(redoLogStatus.get()) +
", switchCounter=" + switchCounter +
", batchSize=" + batchSize +
", millisecondToSleepBetweenMiningQuery=" + millisecondToSleepBetweenMiningQuery +
", keepTransactionsDuration=" + keepTransactionsDuration.get() +
", networkConnectionProblemsCounter" + networkConnectionProblemsCounter +
", batchSizeDefault=" + batchSizeDefault +
", batchSizeMin=" + batchSizeMin +
", batchSizeMax=" + batchSizeMax +
", sleepTimeDefault=" + sleepTimeDefault +
", sleepTimeMin=" + sleepTimeMin +
", sleepTimeMax=" + sleepTimeMax +
", sleepTimeIncrement=" + sleepTimeIncrement +
", totalParseTime=" + totalParseTime +
", totalStartLogMiningSessionDuration=" + totalStartLogMiningSessionDuration +
", lastStartLogMiningSessionDuration=" + lastStartLogMiningSessionDuration +
", maxStartLogMiningSessionDuration=" + maxStartingLogMiningSessionDuration +
", totalProcessTime=" + totalProcessingTime +
", minBatchProcessTime=" + minBatchProcessingTime +
", maxBatchProcessTime=" + maxBatchProcessingTime +
", totalResultSetNextTime=" + totalResultSetNextTime +
", lagFromTheSource=Duration" + lagFromTheSourceDuration.get() +
", maxLagFromTheSourceDuration=" + maxLagFromTheSourceDuration.get() +
", minLagFromTheSourceDuration=" + minLagFromTheSourceDuration.get() +
", lastCommitDuration=" + lastCommitDuration +
", maxCommitDuration=" + maxCommitDuration +
", activeTransactions=" + activeTransactions.get() +
", rolledBackTransactions=" + rolledBackTransactions.get() +
", oversizedTransactions=" + oversizedTransactions.get() +
", committedTransactions=" + committedTransactions.get() +
", abandonedTransactionIds=" + abandonedTransactionIds.get() +
", rolledbackTransactionIds=" + rolledBackTransactionIds.get() +
", registeredDmlCount=" + registeredDmlCount.get() +
", committedDmlCount=" + committedDmlCount.get() +
", errorCount=" + errorCount.get() +
", warningCount=" + warningCount.get() +
", scnFreezeCount=" + scnFreezeCount.get() +
", unparsableDdlCount=" + unparsableDdlCount.get() +
", miningSessionUserGlobalAreaMemory=" + miningSessionUserGlobalAreaMemory.get() +
", miningSessionUserGlobalAreaMaxMemory=" + miningSessionUserGlobalAreaMaxMemory.get() +
", miningSessionProcessGlobalAreaMemory=" + miningSessionProcessGlobalAreaMemory.get() +
", miningSessionProcessGlobalAreaMaxMemory=" + miningSessionProcessGlobalAreaMaxMemory.get() +
'}';
}
}

View File

@ -1,340 +0,0 @@
/*
* Copyright Debezium Authors.
*
* Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0
*/
package io.debezium.connector.oracle;
import java.math.BigInteger;
import java.util.Set;
import io.debezium.pipeline.metrics.StreamingChangeEventSourceMetricsMXBean;
/**
* The JMX exposed interface for Oracle streaming metrics.
*/
public interface OracleStreamingChangeEventSourceMetricsMXBean extends StreamingChangeEventSourceMetricsMXBean {
/**
* @return the current system change number of the database
*/
BigInteger getCurrentScn();
/**
* @return array of current filenames to be used by the mining session.
*/
String[] getCurrentRedoLogFileName();
/**
* @return the minimum number of logs used by a mining session
*/
long getMinimumMinedLogCount();
/**
* @return the maximum number of logs used by a mining session
*/
long getMaximumMinedLogCount();
/**
* Exposes states of redo logs: current, active, inactive, unused ...
* @return array of: (redo log name | status) elements
*/
String[] getRedoLogStatus();
/**
* fetches counter of redo switches for the last day.
* If this number is high , like once in 3 minutes, the troubleshooting on the database level is required.
* @return counter
*/
int getSwitchCounter();
/**
* @return number of captured DML since the connector is up
*/
long getTotalCapturedDmlCount();
/**
* @return average duration of LogMiner view query
*/
Long getMaxDurationOfFetchQueryInMilliseconds();
/**
* LogMiner view query returns number of captured DML , Commit and Rollback. This is what we call a batch.
* @return duration of the last batch fetching
*/
Long getLastDurationOfFetchQueryInMilliseconds();
/**
* LogMiner view query returns number of captured DML , Commit and Rollback. This is what we call a batch.
* @return number of all processed batches
*/
long getFetchingQueryCount();
/**
* @return max number of DML captured during connector start time
*/
Long getMaxCapturedDmlInBatch();
/**
* @return time of processing the last captured batch
*/
long getLastBatchProcessingTimeInMilliseconds();
/**
* @return number of captured DL during last mining session
*/
int getLastCapturedDmlCount();
/**
* Maximum number of entries in LogMiner view to fetch. This is used to set the diapason of the SCN in mining query.
* If difference between "start SCN" and "end SCN" to mine exceeds this limit, end SCN will be set to "start SCN" + batchSize
* @return the limit
*/
int getBatchSize();
/**
* this gives ability to manipulate number of entries in LogMiner view to fetch.
* It has limits to prevent abnormal values
* @param size limit
*/
void setBatchSize(int size);
/**
* @return number of milliseconds for connector to sleep before fetching another batch from the LogMiner view
*/
long getMillisecondToSleepBetweenMiningQuery();
/**
* sets number of milliseconds for connector to sleep before fetching another batch from the LogMiner view
* @param milliseconds to sleep
*/
void setMillisecondToSleepBetweenMiningQuery(long milliseconds);
/**
* change sleeping time
* @param increment true to add, false to deduct
*/
void changeSleepingTime(boolean increment);
void changeBatchSize(boolean increment, boolean lobEnabled);
/**
* This represents the maximum number of entries processed per second from LogMiner sessions.
* Entries include things such as DMLs, commits, rollbacks, etc.
*
* @return the maximum number of entries processed per second from LogMiner sessions
*/
long getMaxBatchProcessingThroughput();
/**
* This represents the average number of entries processed per second from LogMiner sessions.
* Entries include things such as DMLs, commits, rollbacks, etc.
*
* @return the average number of entries per second from LogMiner sessions
*/
long getAverageBatchProcessingThroughput();
/**
* This represents the number of entries processed per second in the last LogMiner session.
* Entries include things such as DMLs, commits, rollbacks, etc.
*
* @return the number of entries processed per second from last LogMiner session
*/
long getLastBatchProcessingThroughput();
/**
* @return the number of connection problems detected
*/
long getNetworkConnectionProblemsCounter();
/**
* @return the total number of milliseconds used to parse DDL/DML statements
*/
long getTotalParseTimeInMilliseconds();
/**
* @return the total number of milliseconds spent starting a log mining session
*/
long getTotalMiningSessionStartTimeInMilliseconds();
/**
* @return the total number of milliseconds the last mining session took to start
*/
long getLastMiningSessionStartTimeInMilliseconds();
/**
* @return the duration in milliseconds of the longest mining session start
*/
long getMaxMiningSessionStartTimeInMilliseconds();
/**
* @return the total number of milliseconds spent mining and processing results
*/
long getTotalProcessingTimeInMilliseconds();
/**
* @return the minimum time in milliseconds spent processing results from a single LogMiner session
*/
long getMinBatchProcessingTimeInMilliseconds();
/**
* @return the maximum time in milliseconds spent processing results from a single LogMiner session
*/
long getMaxBatchProcessingTimeInMilliseconds();
/**
* @return the total number of log miner rows processed.
*/
long getTotalProcessedRows();
/**
* @return the total number of milliseconds spent iterating log miner results calling next.
*/
long getTotalResultSetNextTimeInMilliseconds();
/**
* @return the number of hours to keep transaction in buffer before abandoning
*
* @deprecated Use {@link #getMillisecondsToKeepTransactionsInBuffer()} instead.
*/
@Deprecated
int getHoursToKeepTransactionInBuffer();
/**
* @return the number of milliseconds to keep transactions in the buffer before abandoning
*/
long getMillisecondsToKeepTransactionsInBuffer();
/**
* @return number of current active transactions in the transaction buffer
*/
long getNumberOfActiveTransactions();
/**
* @return the number of committed transactions in the transaction buffer
*/
long getNumberOfCommittedTransactions();
/**
* @return the number of rolled back transactions in the transaction buffer
*/
long getNumberOfRolledBackTransactions();
/**
* @return the number of abandoned transactions because of number of events oversized
*/
long getNumberOfOversizedTransactions();
/**
* @return average number of committed transactions per second in the transaction buffer
*/
long getCommitThroughput();
/**
* @return the number of registered DML operations in the transaction buffer
*/
long getRegisteredDmlCount();
/**
* @return the number of milliseconds of the oldest, active transaction in the buffer.
*/
long getOldestScnAgeInMilliseconds();
/**
* @return the oldest SCN in the transaction buffer
*/
BigInteger getOldestScn();
/**
* @return the last committed SCN from the transaction buffer
*/
BigInteger getCommittedScn();
/**
* @return the current offset SCN
*/
BigInteger getOffsetScn();
/**
* Lag can temporarily be inaccurate on DST changes.
* This is because the timestamps received from LogMiner are in the database local time and do not contain time zone information.
*
* @return lag in milliseconds of latest captured change timestamp from transaction logs and it's placement in the transaction buffer.
*/
long getLagFromSourceInMilliseconds();
/**
* @return maximum lag in milliseconds with the data source
*/
long getMaxLagFromSourceInMilliseconds();
/**
* @return minimum lag in milliseconds with the data source
*/
long getMinLagFromSourceInMilliseconds();
/**
* @return list of abandoned transaction ids from the transaction buffer
*/
Set<String> getAbandonedTransactionIds();
/**
* @return slist of rolled back transaction ids from the transaction buffer
*/
Set<String> getRolledBackTransactionIds();
/**
* @return total duration in milliseconds the last commit operation took in the transaction buffer
*/
long getLastCommitDurationInMilliseconds();
/**
* @return the duration in milliseconds that the longest commit operation took in the transaction buffer
*/
long getMaxCommitDurationInMilliseconds();
/**
* @return the number of errors detected in the connector's log
*/
int getErrorCount();
/**
* @return the number of warnings detected in the connector's log
*/
int getWarningCount();
/**
* @return the number of number of times the SCN does not change and is considered frozen
*/
int getScnFreezeCount();
/**
* @return the number of unparsable ddl statements
*/
int getUnparsableDdlCount();
/**
* @return the current mining session's UGA memory usage in bytes.
*/
long getMiningSessionUserGlobalAreaMemoryInBytes();
/**
* @return the current mining session's UGA maximum memory usage in bytes.
*/
long getMiningSessionUserGlobalAreaMaxMemoryInBytes();
/**
* @return the current mining session's PGA memory usage in bytes.
*/
long getMiningSessionProcessGlobalAreaMemoryInBytes();
/**
* @return the current mining session's PGA maximum memory usage in bytes.
*/
long getMiningSessionProcessGlobalAreaMaxMemoryInBytes();
/**
* Resets metrics.
*/
void reset();
}

View File

@ -8,8 +8,10 @@
import java.sql.SQLException;
import io.debezium.config.Configuration;
import io.debezium.connector.base.ChangeEventQueueMetrics;
import io.debezium.pipeline.ErrorHandler;
import io.debezium.pipeline.EventDispatcher;
import io.debezium.pipeline.source.spi.EventMetadataProvider;
import io.debezium.pipeline.source.spi.StreamingChangeEventSource;
import io.debezium.pipeline.spi.OffsetContext;
import io.debezium.relational.RelationalSnapshotChangeEventSource.RelationalSnapshotContext;
@ -22,7 +24,7 @@
*
* @author Chris Cranford
*/
public interface StreamingAdapter {
public interface StreamingAdapter<T extends AbstractOracleStreamingChangeEventSourceMetrics> {
/**
* Controls whether table names are viewed as case-sensitive or not.
@ -57,7 +59,12 @@ StreamingChangeEventSource<OraclePartition, OracleOffsetContext> getSource(Oracl
OracleDatabaseSchema schema,
OracleTaskContext taskContext,
Configuration jdbcConfig,
OracleStreamingChangeEventSourceMetrics streamingMetrics);
T streamingMetrics);
T getStreamingMetrics(OracleTaskContext taskContext,
ChangeEventQueueMetrics changeEventQueueMetrics,
EventMetadataProvider metadataProvider,
OracleConnectorConfig connectorConfig);
/**
* Returns whether table names are case sensitive.

View File

@ -23,6 +23,7 @@
import io.debezium.DebeziumException;
import io.debezium.config.Configuration;
import io.debezium.connector.base.ChangeEventQueueMetrics;
import io.debezium.connector.oracle.AbstractStreamingAdapter;
import io.debezium.connector.oracle.OracleConnection;
import io.debezium.connector.oracle.OracleConnectorConfig;
@ -30,13 +31,13 @@
import io.debezium.connector.oracle.OracleDatabaseSchema;
import io.debezium.connector.oracle.OracleOffsetContext;
import io.debezium.connector.oracle.OraclePartition;
import io.debezium.connector.oracle.OracleStreamingChangeEventSourceMetrics;
import io.debezium.connector.oracle.OracleTaskContext;
import io.debezium.connector.oracle.Scn;
import io.debezium.document.Document;
import io.debezium.pipeline.ErrorHandler;
import io.debezium.pipeline.EventDispatcher;
import io.debezium.pipeline.source.snapshot.incremental.SignalBasedIncrementalSnapshotContext;
import io.debezium.pipeline.source.spi.EventMetadataProvider;
import io.debezium.pipeline.source.spi.StreamingChangeEventSource;
import io.debezium.pipeline.spi.OffsetContext;
import io.debezium.pipeline.txmetadata.TransactionContext;
@ -50,7 +51,7 @@
/**
* @author Chris Cranford
*/
public class LogMinerAdapter extends AbstractStreamingAdapter {
public class LogMinerAdapter extends AbstractStreamingAdapter<LogMinerStreamingChangeEventSourceMetrics> {
private static final Duration GET_TRANSACTION_SCN_PAUSE = Duration.ofSeconds(1);
@ -92,7 +93,7 @@ public StreamingChangeEventSource<OraclePartition, OracleOffsetContext> getSourc
OracleDatabaseSchema schema,
OracleTaskContext taskContext,
Configuration jdbcConfig,
OracleStreamingChangeEventSourceMetrics streamingMetrics) {
LogMinerStreamingChangeEventSourceMetrics streamingMetrics) {
return new LogMinerStreamingChangeEventSource(
connectorConfig,
connection,
@ -104,6 +105,14 @@ public StreamingChangeEventSource<OraclePartition, OracleOffsetContext> getSourc
streamingMetrics);
}
@Override
public LogMinerStreamingChangeEventSourceMetrics getStreamingMetrics(OracleTaskContext taskContext,
ChangeEventQueueMetrics changeEventQueueMetrics,
EventMetadataProvider metadataProvider,
OracleConnectorConfig connectorConfig) {
return new LogMinerStreamingChangeEventSourceMetrics(taskContext, changeEventQueueMetrics, metadataProvider, connectorConfig);
}
@Override
public OracleOffsetContext determineSnapshotOffset(RelationalSnapshotContext<OraclePartition, OracleOffsetContext> ctx,
OracleConnectorConfig connectorConfig,

View File

@ -35,7 +35,6 @@
import io.debezium.connector.oracle.OracleDatabaseSchema;
import io.debezium.connector.oracle.OracleOffsetContext;
import io.debezium.connector.oracle.OraclePartition;
import io.debezium.connector.oracle.OracleStreamingChangeEventSourceMetrics;
import io.debezium.connector.oracle.Scn;
import io.debezium.connector.oracle.logminer.logwriter.CommitLogWriterFlushStrategy;
import io.debezium.connector.oracle.logminer.logwriter.LogWriterFlushStrategy;
@ -75,7 +74,7 @@ public class LogMinerStreamingChangeEventSource implements StreamingChangeEventS
private final OracleConnectorConfig.LogMiningStrategy strategy;
private final ErrorHandler errorHandler;
private final boolean isContinuousMining;
private final OracleStreamingChangeEventSourceMetrics streamingMetrics;
private final LogMinerStreamingChangeEventSourceMetrics streamingMetrics;
private final OracleConnectorConfig connectorConfig;
private final Duration archiveLogRetention;
private final boolean archiveLogOnlyMode;
@ -90,11 +89,13 @@ public class LogMinerStreamingChangeEventSource implements StreamingChangeEventS
private List<LogFile> currentLogFiles;
private List<BigInteger> currentRedoLogSequences;
private OracleOffsetContext effectiveOffset;
private int currentBatchSize;
private long currentSleepTime;
public LogMinerStreamingChangeEventSource(OracleConnectorConfig connectorConfig,
OracleConnection jdbcConnection, EventDispatcher<OraclePartition, TableId> dispatcher,
ErrorHandler errorHandler, Clock clock, OracleDatabaseSchema schema,
Configuration jdbcConfig, OracleStreamingChangeEventSourceMetrics streamingMetrics) {
Configuration jdbcConfig, LogMinerStreamingChangeEventSourceMetrics streamingMetrics) {
this.jdbcConnection = jdbcConnection;
this.dispatcher = dispatcher;
this.clock = clock;
@ -111,6 +112,11 @@ public LogMinerStreamingChangeEventSource(OracleConnectorConfig connectorConfig,
this.logFileQueryMaxRetries = connectorConfig.getMaximumNumberOfLogQueryRetries();
this.initialDelay = connectorConfig.getLogMiningInitialDelay();
this.maxDelay = connectorConfig.getLogMiningMaxDelay();
this.currentBatchSize = connectorConfig.getLogMiningBatchSizeDefault();
this.currentSleepTime = connectorConfig.getLogMiningSleepTimeDefault().toMillis();
this.streamingMetrics.setBatchSize(this.currentBatchSize);
this.streamingMetrics.setSleepTime(this.currentSleepTime);
}
@Override
@ -176,7 +182,7 @@ public void execute(ChangeEventSourceContext context, OraclePartition partition,
Stopwatch sw = Stopwatch.accumulating().start();
while (context.isRunning()) {
// Calculate time difference before each mining session to detect time zone offset changes (e.g. DST) on database server
streamingMetrics.calculateTimeDifference(getDatabaseSystemTime(jdbcConnection));
streamingMetrics.setDatabaseTimeDifference(getDatabaseSystemTime(jdbcConnection));
if (archiveLogOnlyMode && !waitForStartScnInArchiveLogs(context, startScn)) {
break;
@ -239,7 +245,7 @@ public void execute(ChangeEventSourceContext context, OraclePartition partition,
else {
retryAttempts = 1;
startScn = processor.process(startScn, endScn);
streamingMetrics.setCurrentBatchProcessingTime(Duration.between(start, Instant.now()));
streamingMetrics.setLastBatchProcessingDuration(Duration.between(start, Instant.now()));
captureSessionMemoryStatistics(jdbcConnection);
}
pauseBetweenMiningSessions();
@ -530,8 +536,8 @@ private void updateRedoLogMetrics() throws SQLException {
});
final Set<String> fileNames = getCurrentRedoLogFiles(jdbcConnection);
streamingMetrics.setCurrentLogFileName(fileNames);
streamingMetrics.setRedoLogStatus(logStatuses);
streamingMetrics.setCurrentLogFileNames(fileNames);
streamingMetrics.setRedoLogStatuses(logStatuses);
}
/**
@ -573,7 +579,7 @@ private List<BigInteger> getCurrentRedoLogSequences() throws SQLException {
}
private void pauseBetweenMiningSessions() throws InterruptedException {
Duration period = Duration.ofMillis(streamingMetrics.getMillisecondToSleepBetweenMiningQuery());
Duration period = Duration.ofMillis(streamingMetrics.getSleepTimeInMilliseconds());
Metronome.sleeper(period, clock).pause();
}
@ -628,7 +634,7 @@ public boolean startMiningSession(OracleConnection connection, Scn startScn, Scn
// NOTE: we treat startSCN as the _exclusive_ lower bound for mining,
// whereas START_LOGMNR takes an _inclusive_ lower bound, hence the increment.
connection.executeWithoutCommitting(SqlUtils.startLogMinerStatement(startScn.add(Scn.ONE), endScn, strategy, isContinuousMining));
streamingMetrics.addCurrentMiningSessionStart(Duration.between(start, Instant.now()));
streamingMetrics.setLastMiningSessionStartDuration(Duration.between(start, Instant.now()));
return true;
}
catch (SQLException e) {
@ -672,6 +678,38 @@ public void endMiningSession(OracleConnection connection, OracleOffsetContext of
}
}
private void updateBatchSize(boolean increment) {
if (increment && currentBatchSize < connectorConfig.getLogMiningBatchSizeMin()) {
currentBatchSize += connectorConfig.getLogMiningBatchSizeMin();
if (currentBatchSize == connectorConfig.getLogMiningBatchSizeMax()) {
LOGGER.info("The connector is now using the maximum batch size {} when querying the LogMiner view.{}",
currentBatchSize,
connectorConfig.isLobEnabled() ? "" : " This could be indicate of a large SCN gap.");
}
}
else if (!increment && currentBatchSize > connectorConfig.getLogMiningBatchSizeMin()) {
currentBatchSize -= connectorConfig.getLogMiningBatchSizeMin();
}
if (currentBatchSize != connectorConfig.getLogMiningBatchSizeMax()) {
LOGGER.debug("Updated batch size window, using batch size {}", currentBatchSize);
}
streamingMetrics.setBatchSize(currentBatchSize);
}
private void updateSleepTime(boolean increment) {
if (increment && currentSleepTime < connectorConfig.getLogMiningSleepTimeMax().toMillis()) {
currentSleepTime += connectorConfig.getLogMiningSleepTimeIncrement().toMillis();
}
else if (currentSleepTime > connectorConfig.getLogMiningSleepTimeMin().toMillis()) {
currentSleepTime -= connectorConfig.getLogMiningSleepTimeIncrement().toMillis();
}
LOGGER.debug("Updated sleep time window, using sleep time {}.", currentSleepTime);
streamingMetrics.setSleepTime(currentSleepTime);
}
/**
* Calculates the mining session's end system change number.
*
@ -704,17 +742,17 @@ private Scn calculateEndScn(OracleConnection connection, Scn startScn, Scn prevE
boolean topMiningScnInFarFuture = false;
final Scn defaultBatchScn = Scn.valueOf(connectorConfig.getLogMiningBatchSizeDefault());
if (topScnToMine.subtract(currentScn).compareTo(defaultBatchScn) > 0) {
streamingMetrics.changeBatchSize(false, connectorConfig.isLobEnabled());
updateBatchSize(false);
topMiningScnInFarFuture = true;
}
if (currentScn.subtract(topScnToMine).compareTo(defaultBatchScn) > 0) {
streamingMetrics.changeBatchSize(true, connectorConfig.isLobEnabled());
updateBatchSize(true);
}
// Control sleep time to reduce database impact
if (currentScn.compareTo(topScnToMine) < 0) {
if (!topMiningScnInFarFuture) {
streamingMetrics.changeSleepingTime(true);
updateSleepTime(true);
}
LOGGER.debug("Using current SCN {} as end SCN.", currentScn);
return currentScn;
@ -724,7 +762,7 @@ private Scn calculateEndScn(OracleConnection connection, Scn startScn, Scn prevE
LOGGER.debug("Max batch size too small, using current SCN {} as end SCN.", currentScn);
return currentScn;
}
streamingMetrics.changeSleepingTime(false);
updateSleepTime(false);
if (topScnToMine.compareTo(startScn) < 0) {
LOGGER.debug("Top SCN calculation resulted in end before start SCN, using current SCN {} as end SCN.", currentScn);
return currentScn;

View File

@ -0,0 +1,839 @@
/*
* Copyright Debezium Authors.
*
* Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0
*/
package io.debezium.connector.oracle.logminer;
import java.math.BigInteger;
import java.time.Clock;
import java.time.Duration;
import java.time.Instant;
import java.time.OffsetDateTime;
import java.time.ZoneOffset;
import java.util.Map;
import java.util.Objects;
import java.util.Set;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicLong;
import java.util.concurrent.atomic.AtomicReference;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import io.debezium.annotation.ThreadSafe;
import io.debezium.connector.base.ChangeEventQueueMetrics;
import io.debezium.connector.common.CdcSourceTaskContext;
import io.debezium.connector.oracle.AbstractOracleStreamingChangeEventSourceMetrics;
import io.debezium.connector.oracle.OracleConnectorConfig;
import io.debezium.connector.oracle.Scn;
import io.debezium.pipeline.source.spi.EventMetadataProvider;
import io.debezium.util.LRUCacheMap;
import io.debezium.util.Strings;
/**
* Oracle Streaming Metrics implementation for the Oracle LogMiner streaming adapter.
*
* @author Chris Cranford
*/
@ThreadSafe
public class LogMinerStreamingChangeEventSourceMetrics
extends AbstractOracleStreamingChangeEventSourceMetrics
implements LogMinerStreamingChangeEventSourceMetricsMXBean {
private static final Logger LOGGER = LoggerFactory.getLogger(LogMinerStreamingChangeEventSourceMetrics.class);
private static final long MILLIS_PER_SECOND = 1000L;
private static final int TRANSACTION_ID_SET_SIZE = 10;
private final OracleConnectorConfig connectorConfig;
private final Instant startTime;
private final Clock clock;
private final AtomicReference<Scn> currentScn = new AtomicReference<>(Scn.NULL);
private final AtomicReference<Scn> offsetScn = new AtomicReference<>(Scn.NULL);
private final AtomicReference<Scn> commitScn = new AtomicReference<>(Scn.NULL);
private final AtomicReference<Scn> oldestScn = new AtomicReference<>(Scn.NULL);
private final AtomicReference<Instant> oldestScnTime = new AtomicReference<>();
private final AtomicReference<String[]> currentLogFileNames = new AtomicReference<>(new String[0]);
private final AtomicReference<String[]> redoLogStatuses = new AtomicReference<>(new String[0]);
private final AtomicReference<ZoneOffset> databaseZoneOffset = new AtomicReference<>(ZoneOffset.UTC);
private final AtomicInteger batchSize = new AtomicInteger();
private final AtomicInteger logSwitchCount = new AtomicInteger();
private final AtomicInteger logMinerQueryCount = new AtomicInteger();
private final AtomicLong sleepTime = new AtomicLong();
private final AtomicLong minimumLogsMined = new AtomicLong();
private final AtomicLong maximumLogsMined = new AtomicLong();
private final AtomicLong maxBatchProcessingThroughput = new AtomicLong();
private final AtomicLong timeDifference = new AtomicLong();
private final AtomicLong processedRowsCount = new AtomicLong();
private final AtomicLong activeTransactionCount = new AtomicLong();
private final AtomicLong rolledBackTransactionCount = new AtomicLong();
private final AtomicLong oversizedTransactionCount = new AtomicLong();
private final AtomicLong changesCount = new AtomicLong();
private final AtomicLong scnFreezeCount = new AtomicLong();
private final DurationHistogramMetric batchProcessingDuration = new DurationHistogramMetric();
private final DurationHistogramMetric fetchQueryDuration = new DurationHistogramMetric();
private final DurationHistogramMetric commitDuration = new DurationHistogramMetric();
private final DurationHistogramMetric lagFromSourceDuration = new DurationHistogramMetric();
private final DurationHistogramMetric miningSessionStartupDuration = new DurationHistogramMetric();
private final DurationHistogramMetric parseTimeDuration = new DurationHistogramMetric();
private final DurationHistogramMetric resultSetNextDuration = new DurationHistogramMetric();
private final MaxLongValueMetric userGlobalAreaMemory = new MaxLongValueMetric();
private final MaxLongValueMetric processGlobalAreaMemory = new MaxLongValueMetric();
private final LRUSet<String> abandonedTransactionIds = new LRUSet<>(TRANSACTION_ID_SET_SIZE);
private final LRUSet<String> rolledBackTransactionIds = new LRUSet<>(TRANSACTION_ID_SET_SIZE);
public LogMinerStreamingChangeEventSourceMetrics(CdcSourceTaskContext taskContext,
ChangeEventQueueMetrics changeEventQueueMetrics,
EventMetadataProvider metadataProvider,
OracleConnectorConfig connectorConfig) {
this(taskContext, changeEventQueueMetrics, metadataProvider, connectorConfig, Clock.systemUTC());
}
public LogMinerStreamingChangeEventSourceMetrics(CdcSourceTaskContext taskContext,
ChangeEventQueueMetrics changeEventQueueMetrics,
EventMetadataProvider metadataProvider,
OracleConnectorConfig connectorConfig,
Clock clock) {
super(taskContext, changeEventQueueMetrics, metadataProvider);
this.connectorConfig = connectorConfig;
this.batchSize.set(connectorConfig.getLogMiningBatchSizeDefault());
this.sleepTime.set(connectorConfig.getLogMiningSleepTimeDefault().toMillis());
this.clock = clock;
this.startTime = clock.instant();
reset();
}
@Override
public void reset() {
super.reset();
changesCount.set(0);
processedRowsCount.set(0);
logMinerQueryCount.set(0);
activeTransactionCount.set(0);
rolledBackTransactionCount.set(0);
oversizedTransactionCount.set(0);
scnFreezeCount.set(0);
fetchQueryDuration.reset();
batchProcessingDuration.reset();
parseTimeDuration.reset();
miningSessionStartupDuration.reset();
userGlobalAreaMemory.reset();
processGlobalAreaMemory.reset();
lagFromSourceDuration.reset();
commitDuration.reset();
abandonedTransactionIds.reset();
rolledBackTransactionIds.reset();
oldestScnTime.set(null);
}
@Override
public long getMillisecondsToKeepTransactionsInBuffer() {
return connectorConfig.getLogMiningTransactionRetention().toMillis();
}
@Override
public long getSleepTimeInMilliseconds() {
return sleepTime.get();
}
@Override
public BigInteger getCurrentScn() {
return currentScn.get().asBigInteger();
}
@Override
public BigInteger getOffsetScn() {
return offsetScn.get().asBigInteger();
}
@Override
public BigInteger getCommittedScn() {
return commitScn.get().asBigInteger();
}
@Override
public BigInteger getOldestScn() {
return oldestScn.get().asBigInteger();
}
@Override
public long getOldestScnAgeInMilliseconds() {
if (Objects.isNull(oldestScnTime.get())) {
return 0L;
}
return Duration.between(Instant.now(), oldestScnTime.get()).toMillis();
}
@Override
public String[] getCurrentLogFileNames() {
return currentLogFileNames.get();
}
@Override
public int getBatchSize() {
return batchSize.get();
}
@Override
public long getMinimumMinedLogCount() {
return minimumLogsMined.get();
}
@Override
public long getMaximumMinedLogCount() {
return maximumLogsMined.get();
}
@Override
public String[] getRedoLogStatuses() {
return redoLogStatuses.get();
}
@Override
public int getLogSwitchCount() {
return logSwitchCount.get();
}
@Override
public long getTotalProcessedRows() {
return processedRowsCount.get();
}
@Override
public long getNumberOfActiveTransactions() {
return activeTransactionCount.get();
}
@Override
public long getNumberOfRolledBackTransactions() {
return rolledBackTransactionCount.get();
}
@Override
public long getNumberOfOversizedTransactions() {
return oversizedTransactionCount.get();
}
@Override
public long getTotalChangesCount() {
return changesCount.get();
}
@Override
public long getFetchQueryCount() {
return logMinerQueryCount.get();
}
@Override
public long getScnFreezeCount() {
return scnFreezeCount.get();
}
@Override
public long getLastDurationOfFetchQueryInMilliseconds() {
return fetchQueryDuration.getLast().toMillis();
}
@Override
public long getMaxDurationOfFetchQueryInMilliseconds() {
return fetchQueryDuration.getMaximum().toMillis();
}
@Override
public long getLastBatchProcessingTimeInMilliseconds() {
return batchProcessingDuration.getLast().toMillis();
}
@Override
public long getMinBatchProcessingTimeInMilliseconds() {
return batchProcessingDuration.getMinimum().toMillis();
}
@Override
public long getMaxBatchProcessingTimeInMilliseconds() {
return batchProcessingDuration.getMaximum().toMillis();
}
@Override
public long getTotalBatchProcessingTimeInMilliseconds() {
return batchProcessingDuration.getTotal().toMillis();
}
@Override
public long getCommitThroughput() {
final long timeSpent = Duration.between(startTime, clock.instant()).toMillis();
return getNumberOfCommittedTransactions() * MILLIS_PER_SECOND / (timeSpent != 0 ? timeSpent : 1);
}
@Override
public long getLastBatchProcessingThroughput() {
final Duration lastBatchProcessingDuration = batchProcessingDuration.getLast();
if (lastBatchProcessingDuration.isZero()) {
return 0L;
}
return Math.round(((float) getLastCapturedDmlCount() / lastBatchProcessingDuration.toMillis()) * 1000);
}
@Override
public long getMaxBatchProcessingThroughput() {
return this.maxBatchProcessingThroughput.get();
}
@Override
public long getAverageBatchProcessingThroughput() {
final Duration totalBatchProcessingDuration = batchProcessingDuration.getTotal();
if (totalBatchProcessingDuration.isZero()) {
return 0L;
}
return Math.round(((float) getTotalCapturedDmlCount() / totalBatchProcessingDuration.toMillis()) * 1000);
}
@Override
public long getLastCommitDurationInMilliseconds() {
return commitDuration.getLast().toMillis();
}
@Override
public long getMaxCommitDurationInMilliseconds() {
return commitDuration.getMaximum().toMillis();
}
@Override
public long getLastMiningSessionStartTimeInMilliseconds() {
return miningSessionStartupDuration.getLast().toMillis();
}
@Override
public long getMaxMiningSessionStartTimeInMilliseconds() {
return miningSessionStartupDuration.getMaximum().toMillis();
}
@Override
public long getTotalMiningSessionStartTimeInMilliseconds() {
return miningSessionStartupDuration.getTotal().toMillis();
}
@Override
public long getTotalParseTimeInMilliseconds() {
return parseTimeDuration.getTotal().toMillis();
}
@Override
public long getTotalResultSetNextTimeInMilliseconds() {
return resultSetNextDuration.getTotal().toMillis();
}
@Override
public long getLagFromSourceInMilliseconds() {
return lagFromSourceDuration.getLast().toMillis();
}
@Override
public long getMinLagFromSourceInMilliseconds() {
return lagFromSourceDuration.getMinimum().toMillis();
}
@Override
public long getMaxLagFromSourceInMilliseconds() {
return lagFromSourceDuration.getMaximum().toMillis();
}
@Override
public long getMiningSessionUserGlobalAreaMemoryInBytes() {
return userGlobalAreaMemory.getValue();
}
@Override
public long getMiningSessionUserGlobalAreaMaxMemoryInBytes() {
return userGlobalAreaMemory.getMax();
}
@Override
public long getMiningSessionProcessGlobalAreaMemoryInBytes() {
return processGlobalAreaMemory.getValue();
}
@Override
public long getMiningSessionProcessGlobalAreaMaxMemoryInBytes() {
return processGlobalAreaMemory.getMax();
}
@Override
public Set<String> getAbandonedTransactionIds() {
return abandonedTransactionIds.getAll();
}
@Override
public Set<String> getRolledBackTransactionIds() {
return rolledBackTransactionIds.getAll();
}
/**
* @return database current zone offset
*/
public ZoneOffset getDatabaseOffset() {
return databaseZoneOffset.get();
}
/**
* Set the currently used batch size for querying LogMiner.
*
* @param batchSize batch size used for querying LogMiner
*/
public void setBatchSize(int batchSize) {
this.batchSize.set(batchSize);
}
/**
* Set the connector's currently used sleep/pause time between LogMiner queries.
*
* @param sleepTime sleep time between LogMiner queries
*/
public void setSleepTime(long sleepTime) {
this.sleepTime.set(sleepTime);
}
/**
* Set the current system change number from the database.
*
* @param currentScn database current system change number
*/
public void setCurrentScn(Scn currentScn) {
this.currentScn.set(currentScn);
}
/**
* Set the offset's low-watermark system change number.
*
* @param offsetScn offset's restart system change number, i.e. {@code scn} attribute
*/
public void setOffsetScn(Scn offsetScn) {
this.offsetScn.set(offsetScn);
}
/**
* Sets the offset's high-watermark system change number.
*
* @param commitScn offset's commit system change number, i.e. {@code commit_scn} attribute
*/
// todo: getter is Committed, should this match?
public void setCommitScn(Scn commitScn) {
this.commitScn.set(commitScn);
}
/**
* Sets the details for the oldest system change number in the transaction buffer.
*
* @param oldestScn oldest system change number
* @param changeTime time when the oldest system change number was created
*/
public void setOldestScnDetails(Scn oldestScn, Instant changeTime) {
this.oldestScn.set(oldestScn);
this.oldestScnTime.set(changeTime);
}
/**
* Set the current iteration's logs that are being mined.
*
* @param logFileNames current set of logs that are part of the mining session.
*/
public void setCurrentLogFileNames(Set<String> logFileNames) {
this.currentLogFileNames.set(logFileNames.toArray(String[]::new));
if (logFileNames.size() < minimumLogsMined.get()) {
minimumLogsMined.set(logFileNames.size());
}
else if (minimumLogsMined.get() == 0) {
minimumLogsMined.set(logFileNames.size());
}
if (logFileNames.size() > maximumLogsMined.get()) {
maximumLogsMined.set(logFileNames.size());
}
}
/**
* Set the current logs and their respective statuses.
*
* @param statuses map of file names as key and the file's status as the value.
*/
public void setRedoLogStatuses(Map<String, String> statuses) {
redoLogStatuses.set(statuses.entrySet().stream()
.map(entry -> entry.getKey() + " | " + entry.getValue())
.toArray(String[]::new));
}
/**
* Set the number of log switches in the past day.
*
* @param logSwitchCount number of log switches
*/
public void setSwitchCount(int logSwitchCount) {
this.logSwitchCount.set(logSwitchCount);
}
/**
* Sets the last number of rows processed by the LogMiner mining iteration.
*
* @param processedRowsCount number of rows processed in the last mining iteration
*/
public void setLastProcessedRowsCount(long processedRowsCount) {
this.processedRowsCount.getAndAdd(processedRowsCount);
}
/**
* Sets the number of current, active transactions in the transaction buffer.
*
* @param activeTransactionCount number of active transactions
*/
public void setActiveTransactionCount(long activeTransactionCount) {
this.activeTransactionCount.set(activeTransactionCount);
}
/**
* Increments the number of rolled back transactions.
*/
public void incrementRolledBackTransactionCount() {
rolledBackTransactionCount.incrementAndGet();
}
/**
* Increments the number of over-sized transactions.
*/
public void incrementOversizedTransactionCount() {
oversizedTransactionCount.incrementAndGet();
}
/**
* Increments the total changes seen.
*/
public void incrementTotalChangesCount() {
changesCount.incrementAndGet();
}
/**
* Increments the number of LogMiner queries executed.
*/
public void incrementLogMinerQueryCount() {
logMinerQueryCount.incrementAndGet();
}
/**
* Increments the number of times the system change number is considered frozen and has not
* changed over several consecutive LogMiner query batches.
*/
public void incrementScnFreezeCount() {
scnFreezeCount.incrementAndGet();
}
/**
* Sets the duration of the last LogMiner query execution.
*
* @param duration duration of the last LogMiner query
*/
public void setLastDurationOfFetchQuery(Duration duration) {
fetchQueryDuration.set(duration);
logMinerQueryCount.incrementAndGet();
}
/**
* Sets the duration of the total processing of the last LogMiner query result-set.
*
* @param duration duration of the total processing of the last LogMiner query result-set
*/
public void setLastBatchProcessingDuration(Duration duration) {
batchProcessingDuration.set(duration);
if (getLastBatchProcessingThroughput() > getMaxBatchProcessingThroughput()) {
maxBatchProcessingThroughput.set(getLastBatchProcessingThroughput());
}
}
/**
* Sets the duration of the last transaction commit processing.
*
* @param duration duration of the last transaction commit processing
*/
public void setLastCommitDuration(Duration duration) {
commitDuration.set(duration);
}
/**
* Sets the duration of the last LogMiner mining session start-up and data dictionary load.
*
* @param duration duration of the last LogMiner session start-up
*/
public void setLastMiningSessionStartDuration(Duration duration) {
miningSessionStartupDuration.set(duration);
}
/**
* Sets the duration for parsing the last SQL statement.
*
* @param duration duration for parsing the last SQL statement
*/
public void setLastParseTimeDuration(Duration duration) {
parseTimeDuration.set(duration);
}
/**
* Sets the duration for the last {@code ResultSet#next} function call.
*
* @param duration duration for the last result set next call
*/
public void setLastResultSetNextDuration(Duration duration) {
resultSetNextDuration.set(duration);
}
/**
* Set the database's current user global area (UGA) memory statistics.
*
* @param memory current user global area memory
* @param maxMemory maximum user global area memory
*/
public void setUserGlobalAreaMemory(long memory, long maxMemory) {
userGlobalAreaMemory.setValue(memory);
userGlobalAreaMemory.setMax(maxMemory);
}
/**
* Set the database's current process global area (PGA) memory statistics.
*
* @param memory current process global area memory
* @param maxMemory maximum process global area memory
*/
public void setProcessGlobalAreaMemory(long memory, long maxMemory) {
processGlobalAreaMemory.setValue(memory);
processGlobalAreaMemory.setMax(maxMemory);
}
/**
* Add a transaction to the recently tracked abandoned transactions metric.
*
* @param transactionId transaction identifier
*/
public void addAbandonedTransactionId(String transactionId) {
if (!Strings.isNullOrBlank(transactionId)) {
abandonedTransactionIds.add(transactionId);
}
}
/**
* Add a transaction to the recently rolled back transactions metric.
*
* @param transactionId transaction identifier
*/
public void addRolledBackTransactionId(String transactionId) {
if (!Strings.isNullOrBlank(transactionId)) {
rolledBackTransactionIds.add(transactionId);
}
}
/**
* Sets the database time zone and calculates the difference in time between the database server
* and the connector. These values are necessary to calculate lag metrics.
*
* @param databaseSystemTime the database {@code SYSTIMESTAMP} value
*/
public void setDatabaseTimeDifference(OffsetDateTime databaseSystemTime) {
this.databaseZoneOffset.set(databaseSystemTime.getOffset());
LOGGER.trace("Timezone offset of database time is {} seconds.", databaseZoneOffset.get().getTotalSeconds());
final Instant now = clock.instant();
final long timeDifferenceInMilliseconds = Duration.between(databaseSystemTime.toInstant(), now).toMillis();
this.timeDifference.set(timeDifferenceInMilliseconds);
LOGGER.trace("Current time {} ms, database difference {} ms", now.toEpochMilli(), timeDifferenceInMilliseconds);
}
/**
* Calculates the lag metrics based on the provided event's database change time.
*
* @param changeTime the change time when the database recorded the event
*/
public void calculateLagFromSource(Instant changeTime) {
if (changeTime != null) {
final Instant adjustedTime = changeTime.plusMillis(timeDifference.longValue())
.minusSeconds(databaseZoneOffset.get().getTotalSeconds());
lagFromSourceDuration.set(Duration.between(adjustedTime, clock.instant()).abs());
}
}
@Override
public String toString() {
return "LogMinerStreamingChangeEventSourceMetrics{" +
"connectorConfig=" + connectorConfig +
", startTime=" + startTime +
", clock=" + clock +
", currentScn=" + currentScn +
", offsetScn=" + offsetScn +
", commitScn=" + commitScn +
", oldestScn=" + oldestScn +
", oldestScnTime=" + oldestScnTime +
", currentLogFileNames=" + currentLogFileNames +
", redoLogStatuses=" + redoLogStatuses +
", databaseZoneOffset=" + databaseZoneOffset +
", batchSize=" + batchSize +
", logSwitchCount=" + logSwitchCount +
", logMinerQueryCount=" + logMinerQueryCount +
", sleepTime=" + sleepTime +
", minimumLogsMined=" + minimumLogsMined +
", maximumLogsMined=" + maximumLogsMined +
", maxBatchProcessingThroughput=" + maxBatchProcessingThroughput +
", timeDifference=" + timeDifference +
", processedRowsCount=" + processedRowsCount +
", activeTransactionCount=" + activeTransactionCount +
", rolledBackTransactionCount=" + rolledBackTransactionCount +
", oversizedTransactionCount=" + oversizedTransactionCount +
", changesCount=" + changesCount +
", scnFreezeCount=" + scnFreezeCount +
", batchProcessingDuration=" + batchProcessingDuration +
", fetchQueryDuration=" + fetchQueryDuration +
", commitDuration=" + commitDuration +
", lagFromSourceDuration=" + lagFromSourceDuration +
", miningSessionStartupDuration=" + miningSessionStartupDuration +
", parseTimeDuration=" + parseTimeDuration +
", resultSetNextDuration=" + resultSetNextDuration +
", userGlobalAreaMemory=" + userGlobalAreaMemory +
", processGlobalAreaMemory=" + processGlobalAreaMemory +
", abandonedTransactionIds=" + abandonedTransactionIds +
", rolledBackTransactionIds=" + rolledBackTransactionIds +
"} " + super.toString();
}
/**
* Utility class for tracking histogram-based values for a duration-based metric.
*/
@ThreadSafe
static class DurationHistogramMetric {
private final AtomicReference<Duration> min = new AtomicReference<>(Duration.ZERO);
private final AtomicReference<Duration> max = new AtomicReference<>(Duration.ZERO);
private final AtomicReference<Duration> last = new AtomicReference<>(Duration.ZERO);
private final AtomicReference<Duration> total = new AtomicReference<>(Duration.ZERO);
/**
* Resets the duration metric
*/
void reset() {
min.set(Duration.ZERO);
max.set(Duration.ZERO);
last.set(Duration.ZERO);
total.set(Duration.ZERO);
}
/**
* Sets the last duration-based value for the histogram.
*
* @param lastDuration last duration
*/
void set(Duration lastDuration) {
last.set(lastDuration);
total.accumulateAndGet(lastDuration, Duration::plus);
if (max.get().toMillis() < lastDuration.toMillis()) {
max.set(lastDuration);
}
final long minimumValue = min.get().toMillis();
if (minimumValue > lastDuration.toMillis()) {
min.set(lastDuration);
}
else if (minimumValue == 0L) {
min.set(lastDuration);
}
}
Duration getMinimum() {
return min.get();
}
Duration getMaximum() {
return max.get();
}
Duration getLast() {
return last.get();
}
Duration getTotal() {
return total.get();
}
}
/**
* Utility class for tracking the current and maximum long value.
*/
@ThreadSafe
static class MaxLongValueMetric {
private final AtomicLong value = new AtomicLong();
private final AtomicLong max = new AtomicLong();
public void reset() {
value.set(0L);
max.set(0L);
}
public void setValueAndCalculateMax(long value) {
this.value.set(value);
if (max.get() < value) {
max.set(value);
}
}
public void setValue(long value) {
this.value.set(value);
}
public void setMax(long max) {
if (this.max.get() < max) {
this.max.set(max);
}
}
public long getValue() {
return value.get();
}
public long getMax() {
return max.get();
}
}
/**
* Utility class for maintaining a least-recently-used list of values.
*
* @param <T> the argument type to be stored
*/
@ThreadSafe
static class LRUSet<T> {
private final AtomicReference<LRUCacheMap<T, T>> cache = new AtomicReference<>();
private final int capacity;
LRUSet(int capacity) {
this.cache.set(new LRUCacheMap<>(capacity));
this.capacity = capacity;
}
public void reset() {
this.cache.set(new LRUCacheMap<>(capacity));
}
public void add(T value) {
this.cache.get().put(value, value);
}
public Set<T> getAll() {
return this.cache.get().keySet();
}
}
}

View File

@ -0,0 +1,365 @@
/*
* Copyright Debezium Authors.
*
* Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0
*/
package io.debezium.connector.oracle.logminer;
import java.math.BigInteger;
import java.time.Duration;
import java.util.Set;
import io.debezium.connector.oracle.OracleCommonStreamingChangeEventSourceMetricsMXBean;
/**
* Oracle Streaming Metrics for Oracle LogMiner.
*
* @author Chris Cranford
*/
public interface LogMinerStreamingChangeEventSourceMetricsMXBean
extends OracleCommonStreamingChangeEventSourceMetricsMXBean {
/**
* @return array of currently mined log files
* @deprecated to be removed in Debezium 2.7, replaced by {@link #getCurrentLogFileNames()}
*/
@Deprecated
default String[] getCurrentRedoLogFileName() {
return getCurrentLogFileNames();
}
/**
* @return array of log files and their respective statues in Oracle
* @deprecated to be removed in Debezium 2.7, replaced by {@link #getRedoLogStatuses()}
*/
@Deprecated
default String[] getRedoLogStatus() {
return getRedoLogStatuses();
}
/**
* @return number of log switches observed in the last day
* @deprecated to be removed in Debezium 2.7, replaced by {@link #getLogSwitchCount()}
*/
@Deprecated
default int getSwitchCounter() {
return getLogSwitchCount();
}
/**
* @return number of LogMiner queries executed by the connector
* @deprecated to be removed in Debezium 2.7, replaced by {@link #getFetchQueryCount()}
*/
@Deprecated
default long getFetchingQueryCount() {
return getFetchQueryCount();
}
/**
* @return duration in hours that transactions are retained in the transaction buffer
* @deprecated to be removed in Debezium 2.7, replaced by {@link #getMillisecondsToKeepTransactionsInBuffer()}
*/
@Deprecated
default int getHoursToKeepTransactionInBuffer() {
return (int) Duration.ofMillis(getMillisecondsToKeepTransactionsInBuffer()).toHours();
}
/**
* @return total duration in milliseconds for processing all LogMiner query batches
* @deprecated to be removed in Debezium 2.7, replaced by {@link #getTotalBatchProcessingTimeInMilliseconds()}
*/
@Deprecated
default long getTotalProcessingTimeInMilliseconds() {
return getTotalBatchProcessingTimeInMilliseconds();
}
/**
* @return total number of change events
* @deprecated to be removed in Debezium 2.7, replaced by {@link #getTotalChangesCount()}
*/
@Deprecated
default long getRegisteredDmlCount() {
return getTotalChangesCount();
}
/**
* @return number of milliseconds to sleep between each LogMiner query
* @deprecated to be removed in Debezium 2.7, replaced by {@link #getSleepTimeInMilliseconds()}
*/
@Deprecated
default long getMillisecondsToSleepBetweenMiningQuery() {
return getSleepTimeInMilliseconds();
}
/**
* @return total number of network problems
* @deprecated to be removed in Debezium 2.7 with no replacement
*/
@Deprecated
default long getNetworkConnectionProblemsCounter() {
// This was never used except in tests
return 0L;
}
/**
* Specifies the number of milliseconds that transactions are retained in the transaction buffer
* before the connector discards them due to their age. When set to {@code 0}, transactions are
* retained until they are either committed or rolled back.
*
* @return number of milliseconds that transactions are buffered before being discarded
*/
long getMillisecondsToKeepTransactionsInBuffer();
/**
* @return number of milliseconds that the connector sleeps between LogMiner queries
*/
long getSleepTimeInMilliseconds();
/**
* @return the current system change number of the database
*/
BigInteger getCurrentScn();
/**
* Oracle maintains two watermarks, a low and high system change number watermark. The low
* watermark is the offset system change number, which represents the position in the logs
* where the connector will begin reading changes upon restart.
*
* @return the system change number where the connector will start from on restarts
*/
BigInteger getOffsetScn();
/**
* Oracle maintains two watermarks, a low and high system change number watermark. The high
* watermark is the commit system change number, which represents the position in the logs
* where the last transaction commit occurred. This system change number is used to avoid
* dispatching any transaction that committed before this system change number.
*
* @return the system change number where the connector last observed a commit
*/
BigInteger getCommittedScn();
/**
* @return oldest system change number currently in the transaction buffer
*/
BigInteger getOldestScn();
/**
* @return age in milliseconds of the oldest system change number in the transaction buffer
*/
long getOldestScnAgeInMilliseconds();
/**
* @return array of current filenames to be used by the mining session
*/
String[] getCurrentLogFileNames();
/**
* Specifies the maximum gap between the start and end system change number range used for
* querying changes from LogMiner.
*
* @return the LogMiner query batch size
*/
int getBatchSize();
/**
* @return the minimum number of logs used by a mining session
*/
long getMinimumMinedLogCount();
/**
* @return the maximum number of logs used by a mining session
*/
long getMaximumMinedLogCount();
/**
* Exposes states of redo logs: current, active, inactive, unused ...
* @return array of: (redo log name | status) elements
*/
String[] getRedoLogStatuses();
/**
* @return the number of redo log switches for the current day
*/
int getLogSwitchCount();
/**
* @return the total number of database rows processed from LogMiner
*/
long getTotalProcessedRows();
/**
* @return number of current active transactions in the transaction buffer
*/
long getNumberOfActiveTransactions();
/**
* @return number of transactions seen that were rolled back
*/
long getNumberOfRolledBackTransactions();
/**
* @return number of discarded transactions due to exceeding max event size
*/
long getNumberOfOversizedTransactions();
/**
* @return total number of changes seen by the connector.
*/
long getTotalChangesCount();
/**
* @return number of LogMiner queries executed.
*/
long getFetchQueryCount();
/**
* @return number of times the system change number does not change over consecutive LogMiner queries
*/
long getScnFreezeCount();
/**
* @return duration of the last LogMiner query execution in milliseconds
*/
long getLastDurationOfFetchQueryInMilliseconds();
/**
* @return maximum duration across all LogMiner queries executed in milliseconds
*/
long getMaxDurationOfFetchQueryInMilliseconds();
/**
* @return duration for processing the results of the last LogMiner query in milliseconds
*/
long getLastBatchProcessingTimeInMilliseconds();
/**
* @return minimum duration in milliseconds for processing results from a LogMiner query
*/
long getMinBatchProcessingTimeInMilliseconds();
/**
* @return maximum duration in milliseconds for processing results from a LogMiner query
*/
long getMaxBatchProcessingTimeInMilliseconds();
/**
* @return total duration in milliseconds for processing results for all LogMiner queries
*/
long getTotalBatchProcessingTimeInMilliseconds();
/**
* @return average number of committed transactions per second in the transaction buffer
*/
long getCommitThroughput();
/**
* @return throughput per second for last LogMiner session
*/
long getLastBatchProcessingThroughput();
/**
* @return maximum throughput per second across all LogMiner sessions
*/
long getMaxBatchProcessingThroughput();
/**
* @return average throughput per second across all LogMiner sessions
*/
long getAverageBatchProcessingThroughput();
/**
* @return duration for processing the last transaction commit in milliseconds
*/
long getLastCommitDurationInMilliseconds();
/**
* @return maximum duration for processing a transaction commit in milliseconds
*/
long getMaxCommitDurationInMilliseconds();
/**
* @return duration in milliseconds for the last LogMiner session start-up and data dictionary load
*/
long getLastMiningSessionStartTimeInMilliseconds();
/**
* @return maximum duration in milliseconds for a LogMiner session start-up and data dictionary load
*/
long getMaxMiningSessionStartTimeInMilliseconds();
/**
* @return total duration in milliseconds for all LogMiner session start-ups and data dictionary loads
*/
long getTotalMiningSessionStartTimeInMilliseconds();
/**
* @return total duration in milliseconds for parsing SQL statements
*/
long getTotalParseTimeInMilliseconds();
/**
* Each time a row is processed, the connector makes a call to the underlying JDBC driver to
* fetch the next row and sometimes this fetch may need to make a round-trip to the database
* to get the next batch of rows. This metric tracks the total time spent in milliseconds
* for this particular call over the lifetime of the connector.
*
* @return total duration in milliseconds for all {@code ResultSet#next} calls
*/
long getTotalResultSetNextTimeInMilliseconds();
/**
* Returns the time in milliseconds between when the database captured the change and when the
* change is placed into the transaction buffer by the connector.
*
* @return duration of the lag from the source database in milliseconds
*/
long getLagFromSourceInMilliseconds();
/**
* Returns the minimum time difference in milliseconds between the database capture time and
* the time when the event is placed into the transaction buffer by the connector.
*
* @return minimum duration of the lag from the source database in milliseconds
*/
long getMinLagFromSourceInMilliseconds();
/**
* Returns the maximum time difference in milliseconds between the database capture time and
* the time when the event is placed into the transaction buffer by the connector.
*
* @return maximum duration of the lag from the source database in milliseconds
*/
long getMaxLagFromSourceInMilliseconds();
/**
* @return the current mining session's UGA memory usage in bytes.
*/
long getMiningSessionUserGlobalAreaMemoryInBytes();
/**
* @return the current mining session's UGA maximum memory usage in bytes.
*/
long getMiningSessionUserGlobalAreaMaxMemoryInBytes();
/**
* @return the current mining session's PGA memory usage in bytes.
*/
long getMiningSessionProcessGlobalAreaMemoryInBytes();
/**
* @return the current mining session's PGA maximum memory usage in bytes.
*/
long getMiningSessionProcessGlobalAreaMaxMemoryInBytes();
/**
* @return most recent transaction identifiers that were abandoned
*/
Set<String> getAbandonedTransactionIds();
/**
* @return most recent transaction identifiers that were rolled back
*/
Set<String> getRolledBackTransactionIds();
}

View File

@ -19,8 +19,8 @@
import io.debezium.DebeziumException;
import io.debezium.config.Configuration;
import io.debezium.connector.oracle.OracleConnectorConfig;
import io.debezium.connector.oracle.OracleStreamingChangeEventSourceMetrics;
import io.debezium.connector.oracle.Scn;
import io.debezium.connector.oracle.logminer.LogMinerStreamingChangeEventSourceMetrics;
import io.debezium.jdbc.JdbcConfiguration;
import io.debezium.util.Clock;
import io.debezium.util.Metronome;
@ -42,7 +42,7 @@ public class RacCommitLogWriterFlushStrategy implements LogWriterFlushStrategy {
private static final Logger LOGGER = LoggerFactory.getLogger(RacCommitLogWriterFlushStrategy.class);
private final Map<String, CommitLogWriterFlushStrategy> flushStrategies = new HashMap<>();
private final OracleStreamingChangeEventSourceMetrics streamingMetrics;
private final LogMinerStreamingChangeEventSourceMetrics streamingMetrics;
private final JdbcConfiguration jdbcConfiguration;
private final OracleConnectorConfig connectorConfig;
private final Set<String> hosts;
@ -55,7 +55,7 @@ public class RacCommitLogWriterFlushStrategy implements LogWriterFlushStrategy {
* @param streamingMetrics the streaming metrics, must not be {@code null}
*/
public RacCommitLogWriterFlushStrategy(OracleConnectorConfig connectorConfig, JdbcConfiguration jdbcConfig,
OracleStreamingChangeEventSourceMetrics streamingMetrics) {
LogMinerStreamingChangeEventSourceMetrics streamingMetrics) {
this.jdbcConfiguration = jdbcConfig;
this.streamingMetrics = streamingMetrics;
this.connectorConfig = connectorConfig;

View File

@ -32,10 +32,10 @@
import io.debezium.connector.oracle.OracleOffsetContext;
import io.debezium.connector.oracle.OraclePartition;
import io.debezium.connector.oracle.OracleSchemaChangeEventEmitter;
import io.debezium.connector.oracle.OracleStreamingChangeEventSourceMetrics;
import io.debezium.connector.oracle.Scn;
import io.debezium.connector.oracle.logminer.LogMinerChangeRecordEmitter;
import io.debezium.connector.oracle.logminer.LogMinerQueryBuilder;
import io.debezium.connector.oracle.logminer.LogMinerStreamingChangeEventSourceMetrics;
import io.debezium.connector.oracle.logminer.events.DmlEvent;
import io.debezium.connector.oracle.logminer.events.EventType;
import io.debezium.connector.oracle.logminer.events.LobEraseEvent;
@ -83,7 +83,7 @@ public abstract class AbstractLogMinerEventProcessor<T extends AbstractTransacti
private final OraclePartition partition;
private final OracleOffsetContext offsetContext;
private final EventDispatcher<OraclePartition, TableId> dispatcher;
private final OracleStreamingChangeEventSourceMetrics metrics;
private final LogMinerStreamingChangeEventSourceMetrics metrics;
private final LogMinerDmlParser dmlParser;
private final SelectLobParser selectLobParser;
private final XmlBeginParser xmlBeginParser;
@ -104,7 +104,7 @@ public AbstractLogMinerEventProcessor(ChangeEventSourceContext context,
OraclePartition partition,
OracleOffsetContext offsetContext,
EventDispatcher<OraclePartition, TableId> dispatcher,
OracleStreamingChangeEventSourceMetrics metrics) {
LogMinerStreamingChangeEventSourceMetrics metrics) {
this.context = context;
this.connectorConfig = connectorConfig;
this.schema = schema;
@ -213,7 +213,7 @@ public Scn process(Scn startScn, Scn endScn) throws SQLException, InterruptedExc
Instant queryStart = Instant.now();
try (ResultSet resultSet = statement.executeQuery()) {
metrics.setLastDurationOfBatchCapturing(Duration.between(queryStart, Instant.now()));
metrics.setLastDurationOfFetchQuery(Duration.between(queryStart, Instant.now()));
Instant startProcessTime = Instant.now();
processResults(this.partition, resultSet);
@ -234,7 +234,7 @@ public Scn process(Scn startScn, Scn endScn) throws SQLException, InterruptedExc
LOGGER.debug("Processed in {} ms. Lag: {}. Offset SCN: {}, Offset Commit SCN: {}, Active Transactions: {}, Sleep: {}",
totalTime.toMillis(), metrics.getLagFromSourceInMilliseconds(), offsetContext.getScn(),
offsetContext.getCommitScn(), metrics.getNumberOfActiveTransactions(),
metrics.getMillisecondToSleepBetweenMiningQuery());
metrics.getSleepTimeInMilliseconds());
if (metrics.getNumberOfActiveTransactions() > 0) {
LOGGER.debug("All active transactions: {}",
@ -243,7 +243,7 @@ public Scn process(Scn startScn, Scn endScn) throws SQLException, InterruptedExc
.collect(Collectors.joining(",")));
}
metrics.addProcessedRows(counters.rows);
metrics.setLastProcessedRowsCount(counters.rows);
return calculateNewStartScn(endScn, offsetContext.getCommitScn().getMaxCommittedScn());
}
}
@ -390,7 +390,7 @@ protected void handleStart(LogMinerEventRow row) {
final T transaction = getTransactionCache().get(transactionId);
if (transaction == null && !isRecentlyProcessed(transactionId)) {
getTransactionCache().put(transactionId, createTransaction(row));
metrics.setActiveTransactions(getTransactionCache().size());
metrics.setActiveTransactionCount(getTransactionCache().size());
}
else if (transaction != null && !isRecentlyProcessed(transactionId)) {
LOGGER.trace("Transaction {} is not yet committed and START event detected.", transactionId);
@ -427,13 +427,12 @@ protected void handleCommit(OraclePartition partition, LogMinerEventRow row) thr
final Scn smallestScn;
if (oldestTransaction.isPresent()) {
smallestScn = oldestTransaction.get().getStartScn();
metrics.setOldestScnAge(oldestTransaction.get().getChangeTime());
metrics.setOldestScnDetails(smallestScn, oldestTransaction.get().getChangeTime());
}
else {
smallestScn = Scn.NULL;
metrics.setOldestScnAge(null);
metrics.setOldestScnDetails(Scn.valueOf(-1), null);
}
metrics.setOldestScn(smallestScn.isNull() ? Scn.valueOf(-1) : smallestScn);
final Scn commitScn = row.getScn();
if (offsetContext.getCommitScn().hasCommitAlreadyBeenHandled(row)) {
@ -444,7 +443,7 @@ protected void handleCommit(OraclePartition partition, LogMinerEventRow row) thr
transactionId, offsetContext.getCommitScn(), commitScn, lastCommittedScn);
}
removeTransactionAndEventsFromCache(transaction);
metrics.setActiveTransactions(getTransactionCache().size());
metrics.setActiveTransactionCount(getTransactionCache().size());
return;
}
@ -465,7 +464,7 @@ public void accept(LogMinerEvent event, long eventsProcessed) throws Interrupted
// Update SCN in offset context only if processed SCN less than SCN of other transactions
if (smallestScn.isNull() || commitScn.compareTo(smallestScn) < 0) {
offsetContext.setScn(event.getScn());
metrics.setOldestScn(event.getScn());
metrics.setOldestScnDetails(event.getScn(), event.getChangeTime());
}
offsetContext.setEventScn(event.getScn());
@ -549,15 +548,14 @@ public void accept(LogMinerEvent event, long eventsProcessed) throws Interrupted
dispatcher.dispatchHeartbeatEvent(partition, offsetContext);
}
metrics.calculateLagMetrics(row.getChangeTime());
metrics.calculateLagFromSource(row.getChangeTime());
finalizeTransactionCommit(transactionId, commitScn);
removeTransactionAndEventsFromCache(transaction);
metrics.incrementCommittedTransactions();
metrics.setActiveTransactions(getTransactionCache().size());
metrics.incrementCommittedDmlCount(dispatchedEventCount);
metrics.setCommittedScn(commitScn);
metrics.incrementCommittedTransactionCount();
metrics.setActiveTransactionCount(getTransactionCache().size());
metrics.setCommitScn(commitScn);
metrics.setOffsetScn(offsetContext.getScn());
metrics.setLastCommitDuration(Duration.between(start, Instant.now()));
}
@ -642,8 +640,8 @@ protected void handleRollback(LogMinerEventRow row) {
if (getTransactionCache().containsKey(row.getTransactionId())) {
LOGGER.debug("Transaction {} was rolled back.", row.getTransactionId());
finalizeTransactionRollback(row.getTransactionId(), row.getScn());
metrics.setActiveTransactions(getTransactionCache().size());
metrics.incrementRolledBackTransactions();
metrics.setActiveTransactionCount(getTransactionCache().size());
metrics.incrementRolledBackTransactionCount();
metrics.addRolledBackTransactionId(row.getTransactionId());
counters.rollbackCount++;
}
@ -801,7 +799,7 @@ protected void handleSelectLobLocator(LogMinerEventRow row) {
selectLobParser.isBinary());
});
metrics.incrementRegisteredDmlCount();
metrics.incrementTotalChangesCount();
}
/**
@ -877,8 +875,6 @@ private void handleXmlBegin(LogMinerEventRow row) {
dmlEntry.setObjectOwner(row.getTablespaceName());
return new XmlBeginEvent(row, dmlEntry, xmlBeginParser.getColumnName());
});
metrics.incrementCommittedDmlCount(metrics.getRegisteredDmlCount());
}
private void handleXmlWrite(LogMinerEventRow row) {
@ -1048,7 +1044,7 @@ protected void handleDataEvent(LogMinerEventRow row) throws SQLException, Interr
return new DmlEvent(row, dmlEntry);
});
metrics.incrementRegisteredDmlCount();
metrics.incrementTotalChangesCount();
}
protected void handleUnsupportedEvent(LogMinerEventRow row) {
@ -1114,7 +1110,7 @@ private boolean hasNextWithMetricsUpdate(ResultSet resultSet) throws SQLExceptio
boolean result = false;
try {
if (resultSet.next()) {
metrics.addCurrentResultSetNext(Duration.between(start, Instant.now()));
metrics.setLastResultSetNextDuration(Duration.between(start, Instant.now()));
result = true;
}
@ -1251,7 +1247,7 @@ private LogMinerDmlEntry parseDmlStatement(String redoSql, Table table) {
try {
Instant parseStart = Instant.now();
dmlEntry = dmlParser.parse(redoSql, table);
metrics.addCurrentParseTime(Duration.between(parseStart, Instant.now()));
metrics.setLastParseTimeDuration(Duration.between(parseStart, Instant.now()));
}
catch (DmlParserException e) {
String message = "DML statement couldn't be parsed." +
@ -1367,7 +1363,7 @@ protected void abandonTransactionOverEventThreshold(T transaction) {
LOGGER.warn("Transaction {} exceeds maximum allowed number of events, transaction will be abandoned.", transaction.getTransactionId());
metrics.incrementWarningCount();
getAndRemoveTransactionFromCache(transaction.getTransactionId());
metrics.incrementOversizedTransactions();
metrics.incrementOversizedTransactionCount();
}
/**

View File

@ -9,6 +9,7 @@
import java.sql.ResultSet;
import java.sql.SQLException;
import java.time.Duration;
import java.time.Instant;
import java.util.Comparator;
import java.util.Iterator;
import java.util.List;
@ -24,8 +25,8 @@
import io.debezium.connector.oracle.OracleDatabaseSchema;
import io.debezium.connector.oracle.OracleOffsetContext;
import io.debezium.connector.oracle.OraclePartition;
import io.debezium.connector.oracle.OracleStreamingChangeEventSourceMetrics;
import io.debezium.connector.oracle.Scn;
import io.debezium.connector.oracle.logminer.LogMinerStreamingChangeEventSourceMetrics;
import io.debezium.connector.oracle.logminer.events.LogMinerEvent;
import io.debezium.connector.oracle.logminer.events.LogMinerEventRow;
import io.debezium.connector.oracle.logminer.processor.AbstractLogMinerEventProcessor;
@ -45,7 +46,7 @@ public abstract class AbstractInfinispanLogMinerEventProcessor extends AbstractL
private static final Logger LOGGER = LoggerFactory.getLogger(AbstractInfinispanLogMinerEventProcessor.class);
private final OracleConnection jdbcConnection;
private final OracleStreamingChangeEventSourceMetrics metrics;
private final LogMinerStreamingChangeEventSourceMetrics metrics;
private final OraclePartition partition;
private final OracleOffsetContext offsetContext;
private final EventDispatcher<OraclePartition, TableId> dispatcher;
@ -57,10 +58,10 @@ public AbstractInfinispanLogMinerEventProcessor(ChangeEventSourceContext context
OraclePartition partition,
OracleOffsetContext offsetContext,
OracleDatabaseSchema schema,
OracleStreamingChangeEventSourceMetrics metrics) {
LogMinerStreamingChangeEventSourceMetrics metrics) {
super(context, connectorConfig, schema, partition, offsetContext, dispatcher, metrics);
this.jdbcConnection = jdbcConnection;
this.metrics = metrics;
this.metrics = (LogMinerStreamingChangeEventSourceMetrics) metrics;
this.partition = partition;
this.offsetContext = offsetContext;
this.dispatcher = dispatcher;
@ -268,11 +269,11 @@ protected void addToTransaction(String transactionId, LogMinerEventRow row, Supp
// Add new event at eventId offset
LOGGER.trace("Transaction {}, adding event reference at key {}", transactionId, eventKey);
getEventCache().put(eventKey, eventSupplier.get());
metrics.calculateLagMetrics(row.getChangeTime());
metrics.calculateLagFromSource(row.getChangeTime());
}
// When using Infinispan, this extra put is required so that the state is properly synchronized
getTransactionCache().put(transactionId, transaction);
metrics.setActiveTransactions(getTransactionCache().size());
metrics.setActiveTransactionCount(getTransactionCache().size());
}
else {
LOGGER.warn("Event for transaction {} skipped as transaction has been processed.", transactionId);
@ -303,11 +304,14 @@ protected Scn calculateNewStartScn(Scn endScn, Scn maxCommittedScn) throws Inter
// Cleanup caches based on current state of the transaction cache
final Optional<InfinispanTransaction> oldestTransaction = getOldestTransactionInCache();
final Scn minCacheScn;
final Instant minCacheScnChangeTime;
if (oldestTransaction.isPresent()) {
minCacheScn = oldestTransaction.get().getStartScn();
minCacheScnChangeTime = oldestTransaction.get().getChangeTime();
}
else {
minCacheScn = Scn.NULL;
minCacheScnChangeTime = null;
}
if (!minCacheScn.isNull()) {
@ -342,7 +346,7 @@ protected Scn calculateNewStartScn(Scn endScn, Scn maxCommittedScn) throws Inter
// update offsets
offsetContext.setScn(endScn);
metrics.setOldestScn(minCacheScn);
metrics.setOldestScnDetails(minCacheScn, minCacheScnChangeTime);
metrics.setOffsetScn(endScn);
// optionally dispatch a heartbeat event

View File

@ -35,8 +35,8 @@
import io.debezium.connector.oracle.OracleDatabaseSchema;
import io.debezium.connector.oracle.OracleOffsetContext;
import io.debezium.connector.oracle.OraclePartition;
import io.debezium.connector.oracle.OracleStreamingChangeEventSourceMetrics;
import io.debezium.connector.oracle.Scn;
import io.debezium.connector.oracle.logminer.LogMinerStreamingChangeEventSourceMetrics;
import io.debezium.connector.oracle.logminer.events.LogMinerEvent;
import io.debezium.pipeline.EventDispatcher;
import io.debezium.pipeline.source.spi.ChangeEventSource.ChangeEventSourceContext;
@ -70,7 +70,7 @@ public EmbeddedInfinispanLogMinerEventProcessor(ChangeEventSourceContext context
OraclePartition partition,
OracleOffsetContext offsetContext,
OracleDatabaseSchema schema,
OracleStreamingChangeEventSourceMetrics metrics) {
LogMinerStreamingChangeEventSourceMetrics metrics) {
super(context, connectorConfig, jdbcConnection, dispatcher, partition, offsetContext, schema, metrics);
LOGGER.info("Using Infinispan in embedded mode.");

View File

@ -33,8 +33,8 @@
import io.debezium.connector.oracle.OracleDatabaseSchema;
import io.debezium.connector.oracle.OracleOffsetContext;
import io.debezium.connector.oracle.OraclePartition;
import io.debezium.connector.oracle.OracleStreamingChangeEventSourceMetrics;
import io.debezium.connector.oracle.Scn;
import io.debezium.connector.oracle.logminer.LogMinerStreamingChangeEventSourceMetrics;
import io.debezium.connector.oracle.logminer.events.LogMinerEvent;
import io.debezium.connector.oracle.logminer.processor.infinispan.marshalling.LogMinerEventMarshallerImpl;
import io.debezium.connector.oracle.logminer.processor.infinispan.marshalling.TransactionMarshallerImpl;
@ -75,7 +75,7 @@ public RemoteInfinispanLogMinerEventProcessor(ChangeEventSourceContext context,
OraclePartition partition,
OracleOffsetContext offsetContext,
OracleDatabaseSchema schema,
OracleStreamingChangeEventSourceMetrics metrics) {
LogMinerStreamingChangeEventSourceMetrics metrics) {
super(context, connectorConfig, jdbcConnection, dispatcher, partition, offsetContext, schema, metrics);
Configuration config = new ConfigurationBuilder()

View File

@ -27,8 +27,8 @@
import io.debezium.connector.oracle.OracleDatabaseSchema;
import io.debezium.connector.oracle.OracleOffsetContext;
import io.debezium.connector.oracle.OraclePartition;
import io.debezium.connector.oracle.OracleStreamingChangeEventSourceMetrics;
import io.debezium.connector.oracle.Scn;
import io.debezium.connector.oracle.logminer.LogMinerStreamingChangeEventSourceMetrics;
import io.debezium.connector.oracle.logminer.SqlUtils;
import io.debezium.connector.oracle.logminer.events.LogMinerEvent;
import io.debezium.connector.oracle.logminer.events.LogMinerEventRow;
@ -53,7 +53,7 @@ public class MemoryLogMinerEventProcessor extends AbstractLogMinerEventProcessor
private final EventDispatcher<OraclePartition, TableId> dispatcher;
private final OraclePartition partition;
private final OracleOffsetContext offsetContext;
private final OracleStreamingChangeEventSourceMetrics metrics;
private final LogMinerStreamingChangeEventSourceMetrics metrics;
/**
* Cache of transactions, keyed based on the transaction's unique identifier
@ -73,13 +73,13 @@ public MemoryLogMinerEventProcessor(ChangeEventSourceContext context,
OraclePartition partition,
OracleOffsetContext offsetContext,
OracleDatabaseSchema schema,
OracleStreamingChangeEventSourceMetrics metrics) {
LogMinerStreamingChangeEventSourceMetrics metrics) {
super(context, connectorConfig, schema, partition, offsetContext, dispatcher, metrics);
this.jdbcConnection = jdbcConnection;
this.dispatcher = dispatcher;
this.partition = partition;
this.offsetContext = offsetContext;
this.metrics = metrics;
this.metrics = (LogMinerStreamingChangeEventSourceMetrics) metrics;
}
@Override
@ -160,19 +160,18 @@ public void abandonTransactions(Duration retention) throws InterruptedException
iterator.remove();
metrics.addAbandonedTransactionId(entry.getKey());
metrics.setActiveTransactions(transactionCache.size());
metrics.setActiveTransactionCount(transactionCache.size());
}
}
// Update the oldest scn metric are transaction abandonment
final Optional<MemoryTransaction> oldestTransaction = getOldestTransactionInCache();
if (oldestTransaction.isPresent()) {
metrics.setOldestScn(oldestTransaction.get().getStartScn());
metrics.setOldestScnAge(oldestTransaction.get().getChangeTime());
final MemoryTransaction transaction = oldestTransaction.get();
metrics.setOldestScnDetails(transaction.getStartScn(), transaction.getChangeTime());
}
else {
metrics.setOldestScn(Scn.NULL);
metrics.setOldestScnAge(null);
metrics.setOldestScnDetails(Scn.NULL, null);
}
offsetContext.setScn(thresholdScn);
@ -273,10 +272,10 @@ protected void addToTransaction(String transactionId, LogMinerEventRow row, Supp
// Add new event at eventId offset
LOGGER.trace("Transaction {}, adding event reference at index {}", transactionId, eventId);
transaction.getEvents().add(eventSupplier.get());
metrics.calculateLagMetrics(row.getChangeTime());
metrics.calculateLagFromSource(row.getChangeTime());
}
metrics.setActiveTransactions(getTransactionCache().size());
metrics.setActiveTransactionCount(getTransactionCache().size());
}
else if (!getConfig().isLobEnabled()) {
// Explicitly only log this warning when LobEnabled is false because its commonplace for a

View File

@ -13,13 +13,13 @@
import org.slf4j.LoggerFactory;
import io.debezium.config.Configuration;
import io.debezium.connector.base.ChangeEventQueueMetrics;
import io.debezium.connector.oracle.AbstractStreamingAdapter;
import io.debezium.connector.oracle.OracleConnection;
import io.debezium.connector.oracle.OracleConnectorConfig;
import io.debezium.connector.oracle.OracleDatabaseSchema;
import io.debezium.connector.oracle.OracleOffsetContext;
import io.debezium.connector.oracle.OraclePartition;
import io.debezium.connector.oracle.OracleStreamingChangeEventSourceMetrics;
import io.debezium.connector.oracle.OracleTaskContext;
import io.debezium.connector.oracle.OracleValueConverters;
import io.debezium.connector.oracle.Scn;
@ -27,6 +27,7 @@
import io.debezium.pipeline.ErrorHandler;
import io.debezium.pipeline.EventDispatcher;
import io.debezium.pipeline.source.snapshot.incremental.SignalBasedIncrementalSnapshotContext;
import io.debezium.pipeline.source.spi.EventMetadataProvider;
import io.debezium.pipeline.source.spi.StreamingChangeEventSource;
import io.debezium.pipeline.spi.OffsetContext;
import io.debezium.pipeline.txmetadata.TransactionContext;
@ -40,7 +41,7 @@
*
* @author Chris Cranford
*/
public class OpenLogReplicatorAdapter extends AbstractStreamingAdapter {
public class OpenLogReplicatorAdapter extends AbstractStreamingAdapter<OpenLogReplicatorStreamingChangeEventSourceMetrics> {
private static final Logger LOGGER = LoggerFactory.getLogger(OpenLogReplicatorAdapter.class);
private static final String TYPE = "olr";
@ -77,7 +78,7 @@ public StreamingChangeEventSource<OraclePartition, OracleOffsetContext> getSourc
OracleDatabaseSchema schema,
OracleTaskContext taskContext,
Configuration jdbcConfig,
OracleStreamingChangeEventSourceMetrics streamingMetrics) {
OpenLogReplicatorStreamingChangeEventSourceMetrics streamingMetrics) {
return new OpenLogReplicatorStreamingChangeEventSource(
connectorConfig,
connection,
@ -88,6 +89,14 @@ public StreamingChangeEventSource<OraclePartition, OracleOffsetContext> getSourc
streamingMetrics);
}
@Override
public OpenLogReplicatorStreamingChangeEventSourceMetrics getStreamingMetrics(OracleTaskContext taskContext,
ChangeEventQueueMetrics changeEventQueueMetrics,
EventMetadataProvider metadataProvider,
OracleConnectorConfig connectorConfig) {
return new OpenLogReplicatorStreamingChangeEventSourceMetrics(taskContext, changeEventQueueMetrics, metadataProvider);
}
@Override
public OracleOffsetContext determineSnapshotOffset(RelationalSnapshotContext<OraclePartition, OracleOffsetContext> ctx,
OracleConnectorConfig connectorConfig, OracleConnection connection)

View File

@ -22,7 +22,6 @@
import io.debezium.connector.oracle.OracleOffsetContext;
import io.debezium.connector.oracle.OraclePartition;
import io.debezium.connector.oracle.OracleSchemaChangeEventEmitter;
import io.debezium.connector.oracle.OracleStreamingChangeEventSourceMetrics;
import io.debezium.connector.oracle.OracleValueConverters;
import io.debezium.connector.oracle.Scn;
import io.debezium.connector.oracle.olr.client.OlrNetworkClient;
@ -59,7 +58,7 @@ public class OpenLogReplicatorStreamingChangeEventSource implements StreamingCha
private final ErrorHandler errorHandler;
private final Clock clock;
private final OracleDatabaseSchema schema;
private final OracleStreamingChangeEventSourceMetrics streamingMetrics;
private final OpenLogReplicatorStreamingChangeEventSourceMetrics streamingMetrics;
private OlrNetworkClient client;
private OraclePartition partition;
@ -72,7 +71,7 @@ public OpenLogReplicatorStreamingChangeEventSource(OracleConnectorConfig connect
EventDispatcher<OraclePartition, TableId> dispatcher,
ErrorHandler errorHandler, Clock clock,
OracleDatabaseSchema schema,
OracleStreamingChangeEventSourceMetrics streamingMetrics) {
OpenLogReplicatorStreamingChangeEventSourceMetrics streamingMetrics) {
this.connectorConfig = connectorConfig;
this.dispatcher = dispatcher;
this.jdbcConnection = connection;
@ -185,6 +184,9 @@ private void onEvent(StreamingEvent event) throws Exception {
throw new DebeziumException("Unexpected event type detected: " + payloadEvent.getType());
}
}
streamingMetrics.incrementProcessedEventsCount();
streamingMetrics.setCheckpointDetails(event.getCheckpointScn(), event.getCheckpointIndex());
}
private void onBeginEvent(StreamingEvent event) {
@ -195,10 +197,6 @@ private void onBeginEvent(StreamingEvent event) {
offsetContext.setSourceTime(event.getTimestamp());
transactionEvents = false;
streamingMetrics.setOffsetScn(offsetContext.getScn());
streamingMetrics.setActiveTransactions(1);
streamingMetrics.addProcessedRows(1L);
// We do not specifically start a transaction boundary here.
//
// This is delayed until the data change event on the first data change that is to be
@ -213,11 +211,7 @@ private void onCommitEvent(StreamingEvent event) throws InterruptedException {
offsetContext.setTransactionId(event.getXid());
offsetContext.setSourceTime(event.getTimestamp());
streamingMetrics.setOffsetScn(offsetContext.getScn());
streamingMetrics.setCommittedScn(offsetContext.getScn());
streamingMetrics.setActiveTransactions(0);
streamingMetrics.addProcessedRows(1L);
streamingMetrics.incrementCommittedTransactions();
streamingMetrics.incrementCommittedTransactionCount();
// We may see empty transactions and in this case we don't want to emit a transaction boundary
// record for these cases. Only trigger commit when there are valid changes.
@ -242,9 +236,6 @@ private void onCheckpointEvent(StreamingEvent event) throws InterruptedException
offsetContext.setTransactionId(event.getXid());
offsetContext.setSourceTime(event.getTimestamp());
streamingMetrics.setOffsetScn(offsetContext.getScn());
streamingMetrics.setCommittedScn(offsetContext.getScn());
// For checkpoints, we do not emit any type of normal event, so while we do update
// the checkpoint details, these won't be flushed until the next commit flush.
// If the environment has low activity, enabling heartbeats will guarantee that
@ -291,10 +282,7 @@ private void onMutationEvent(StreamingEvent event, AbstractMutationEvent mutatio
offsetContext.setTransactionId(event.getXid());
offsetContext.tableEvent(tableId, event.getTimestamp());
streamingMetrics.setOffsetScn(offsetContext.getScn());
streamingMetrics.addProcessedRows(1L);
streamingMetrics.setLastCapturedDmlCount(1);
streamingMetrics.incrementRegisteredDmlCount();
updateCheckpoint(event);
@ -342,10 +330,6 @@ else if (tableId.table().startsWith("BIN$") && tableId.table().endsWith("==$0"))
offsetContext.setTransactionId(event.getXid());
offsetContext.tableEvent(tableId, event.getTimestamp());
streamingMetrics.setOffsetScn(offsetContext.getScn());
streamingMetrics.setCommittedScn(offsetContext.getScn());
streamingMetrics.addProcessedRows(1L);
final String sqlStatement = schemaEvent.getSql().toLowerCase().trim();
// todo: do we want to let other ddl statements be emitted for non-tables?

View File

@ -0,0 +1,60 @@
/*
* Copyright Debezium Authors.
*
* Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0
*/
package io.debezium.connector.oracle.olr;
import java.math.BigInteger;
import java.util.concurrent.atomic.AtomicLong;
import java.util.concurrent.atomic.AtomicReference;
import io.debezium.connector.base.ChangeEventQueueMetrics;
import io.debezium.connector.common.CdcSourceTaskContext;
import io.debezium.connector.oracle.AbstractOracleStreamingChangeEventSourceMetrics;
import io.debezium.connector.oracle.Scn;
import io.debezium.pipeline.source.spi.EventMetadataProvider;
/**
* Oracle Streaming Metrics implementation for the Oracle OpenLogReplicator adapter.
*
* @author Chris Cranford
*/
public class OpenLogReplicatorStreamingChangeEventSourceMetrics
extends AbstractOracleStreamingChangeEventSourceMetrics
implements OpenLogReplicatorStreamingChangeEventSourceMetricsMXBean {
private final AtomicReference<Scn> checkpointScn = new AtomicReference<>(Scn.NULL);
private final AtomicLong checkpointIndex = new AtomicLong();
private final AtomicLong processedEventsCount = new AtomicLong();
public OpenLogReplicatorStreamingChangeEventSourceMetrics(CdcSourceTaskContext taskContext,
ChangeEventQueueMetrics changeEventQueueMetrics,
EventMetadataProvider metadataProvider) {
super(taskContext, changeEventQueueMetrics, metadataProvider);
}
@Override
public BigInteger getCheckpointScn() {
return checkpointScn.get().asBigInteger();
}
@Override
public long getCheckpointIndex() {
return checkpointIndex.get();
}
@Override
public long getProcessedEventCount() {
return processedEventsCount.get();
}
public void setCheckpointDetails(Scn checkpointScn, Long checkpointIndex) {
this.checkpointScn.set(checkpointScn);
this.checkpointIndex.set(checkpointIndex);
}
public void incrementProcessedEventsCount() {
processedEventsCount.incrementAndGet();
}
}

View File

@ -0,0 +1,35 @@
/*
* Copyright Debezium Authors.
*
* Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0
*/
package io.debezium.connector.oracle.olr;
import java.math.BigInteger;
import io.debezium.connector.oracle.OracleCommonStreamingChangeEventSourceMetricsMXBean;
/**
* Oracle Streaming Metrics for OpenLogReplicator.
*
* @author Chris Cranford
*/
public interface OpenLogReplicatorStreamingChangeEventSourceMetricsMXBean
extends OracleCommonStreamingChangeEventSourceMetricsMXBean {
/**
* @return checkpoint scn where the connector resumes on restart
*/
BigInteger getCheckpointScn();
/**
* @return checkpoint index, resume position within a checkpoint block
*/
long getCheckpointIndex();
/**
* @return number of events processed from OpenLogReplicator
*/
long getProcessedEventCount();
}

View File

@ -22,7 +22,6 @@
import io.debezium.connector.oracle.OracleOffsetContext;
import io.debezium.connector.oracle.OraclePartition;
import io.debezium.connector.oracle.OracleSchemaChangeEventEmitter;
import io.debezium.connector.oracle.OracleStreamingChangeEventSourceMetrics;
import io.debezium.connector.oracle.OracleValueConverters;
import io.debezium.connector.oracle.xstream.XstreamStreamingChangeEventSource.PositionAndScn;
import io.debezium.pipeline.ErrorHandler;
@ -60,7 +59,7 @@ class LcrEventHandler implements XStreamLCRCallbackHandler {
private final OracleOffsetContext offsetContext;
private final boolean tablenameCaseInsensitive;
private final XstreamStreamingChangeEventSource eventSource;
private final OracleStreamingChangeEventSourceMetrics streamingMetrics;
private final XStreamStreamingChangeEventSourceMetrics streamingMetrics;
private final Map<String, ChunkColumnValues> columnChunks;
private RowLCR currentRow;
@ -68,7 +67,7 @@ class LcrEventHandler implements XStreamLCRCallbackHandler {
EventDispatcher<OraclePartition, TableId> dispatcher, Clock clock,
OracleDatabaseSchema schema, OraclePartition partition, OracleOffsetContext offsetContext,
boolean tablenameCaseInsensitive, XstreamStreamingChangeEventSource eventSource,
OracleStreamingChangeEventSourceMetrics streamingMetrics) {
XStreamStreamingChangeEventSourceMetrics streamingMetrics) {
this.connectorConfig = connectorConfig;
this.errorHandler = errorHandler;
this.dispatcher = dispatcher;

View File

@ -13,13 +13,13 @@
import org.slf4j.LoggerFactory;
import io.debezium.config.Configuration;
import io.debezium.connector.base.ChangeEventQueueMetrics;
import io.debezium.connector.oracle.AbstractStreamingAdapter;
import io.debezium.connector.oracle.OracleConnection;
import io.debezium.connector.oracle.OracleConnectorConfig;
import io.debezium.connector.oracle.OracleDatabaseSchema;
import io.debezium.connector.oracle.OracleOffsetContext;
import io.debezium.connector.oracle.OraclePartition;
import io.debezium.connector.oracle.OracleStreamingChangeEventSourceMetrics;
import io.debezium.connector.oracle.OracleTaskContext;
import io.debezium.connector.oracle.Scn;
import io.debezium.connector.oracle.SourceInfo;
@ -27,6 +27,7 @@
import io.debezium.pipeline.ErrorHandler;
import io.debezium.pipeline.EventDispatcher;
import io.debezium.pipeline.source.snapshot.incremental.SignalBasedIncrementalSnapshotContext;
import io.debezium.pipeline.source.spi.EventMetadataProvider;
import io.debezium.pipeline.source.spi.StreamingChangeEventSource;
import io.debezium.pipeline.spi.OffsetContext;
import io.debezium.pipeline.txmetadata.TransactionContext;
@ -40,7 +41,7 @@
*
* @author Chris Cranford
*/
public class XStreamAdapter extends AbstractStreamingAdapter {
public class XStreamAdapter extends AbstractStreamingAdapter<XStreamStreamingChangeEventSourceMetrics> {
private static final Logger LOGGER = LoggerFactory.getLogger(XStreamAdapter.class);
@ -85,7 +86,7 @@ public StreamingChangeEventSource<OraclePartition, OracleOffsetContext> getSourc
OracleDatabaseSchema schema,
OracleTaskContext taskContext,
Configuration jdbcConfig,
OracleStreamingChangeEventSourceMetrics streamingMetrics) {
XStreamStreamingChangeEventSourceMetrics streamingMetrics) {
return new XstreamStreamingChangeEventSource(
connectorConfig,
connection,
@ -96,6 +97,14 @@ public StreamingChangeEventSource<OraclePartition, OracleOffsetContext> getSourc
streamingMetrics);
}
@Override
public XStreamStreamingChangeEventSourceMetrics getStreamingMetrics(OracleTaskContext taskContext,
ChangeEventQueueMetrics changeEventQueueMetrics,
EventMetadataProvider metadataProvider,
OracleConnectorConfig connectorConfig) {
return new XStreamStreamingChangeEventSourceMetrics(taskContext, changeEventQueueMetrics, metadataProvider);
}
@Override
public TableNameCaseSensitivity getTableNameCaseSensitivity(OracleConnection connection) {
// Always use tablename case insensitivity true when on Oracle 11, otherwise false.

View File

@ -0,0 +1,24 @@
/*
* Copyright Debezium Authors.
*
* Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0
*/
package io.debezium.connector.oracle.xstream;
import io.debezium.connector.base.ChangeEventQueueMetrics;
import io.debezium.connector.common.CdcSourceTaskContext;
import io.debezium.connector.oracle.AbstractOracleStreamingChangeEventSourceMetrics;
import io.debezium.pipeline.source.spi.EventMetadataProvider;
/**
* Oracle Streaming Metrics implementation for the Oracle XStream streaming adapter.
*
* @author Chris Cranford
*/
public class XStreamStreamingChangeEventSourceMetrics extends AbstractOracleStreamingChangeEventSourceMetrics {
public XStreamStreamingChangeEventSourceMetrics(CdcSourceTaskContext taskContext,
ChangeEventQueueMetrics changeEventQueueMetrics,
EventMetadataProvider metadataProvider) {
super(taskContext, changeEventQueueMetrics, metadataProvider);
}
}

View File

@ -0,0 +1,17 @@
/*
* Copyright Debezium Authors.
*
* Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0
*/
package io.debezium.connector.oracle.xstream;
import io.debezium.connector.oracle.OracleCommonStreamingChangeEventSourceMetricsMXBean;
/**
* Oracle Streaming Metrics for Oracle XStream.
*
* @author Chris Cranford
*/
public interface XStreamStreamingChangeEventSourceMetricsMXBean
extends OracleCommonStreamingChangeEventSourceMetricsMXBean {
}

View File

@ -19,7 +19,6 @@
import io.debezium.connector.oracle.OracleDatabaseVersion;
import io.debezium.connector.oracle.OracleOffsetContext;
import io.debezium.connector.oracle.OraclePartition;
import io.debezium.connector.oracle.OracleStreamingChangeEventSourceMetrics;
import io.debezium.connector.oracle.Scn;
import io.debezium.connector.oracle.SourceInfo;
import io.debezium.connector.oracle.StreamingAdapter.TableNameCaseSensitivity;
@ -52,7 +51,7 @@ public class XstreamStreamingChangeEventSource implements StreamingChangeEventSo
private final ErrorHandler errorHandler;
private final Clock clock;
private final OracleDatabaseSchema schema;
private final OracleStreamingChangeEventSourceMetrics streamingMetrics;
private final XStreamStreamingChangeEventSourceMetrics streamingMetrics;
private final String xStreamServerName;
private volatile XStreamOut xsOut;
private final int posVersion;
@ -69,7 +68,7 @@ public class XstreamStreamingChangeEventSource implements StreamingChangeEventSo
public XstreamStreamingChangeEventSource(OracleConnectorConfig connectorConfig, OracleConnection jdbcConnection,
EventDispatcher<OraclePartition, TableId> dispatcher, ErrorHandler errorHandler,
Clock clock, OracleDatabaseSchema schema,
OracleStreamingChangeEventSourceMetrics streamingMetrics) {
XStreamStreamingChangeEventSourceMetrics streamingMetrics) {
this.connectorConfig = connectorConfig;
this.jdbcConnection = jdbcConnection;
this.dispatcher = dispatcher;

View File

@ -7,22 +7,16 @@
import static io.debezium.config.CommonConnectorConfig.DEFAULT_MAX_BATCH_SIZE;
import static io.debezium.config.CommonConnectorConfig.DEFAULT_MAX_QUEUE_SIZE;
import static org.assertj.core.api.Assertions.assertThat;
import static org.mockito.Mockito.mock;
import java.time.Clock;
import java.time.Duration;
import java.time.Instant;
import java.time.OffsetDateTime;
import java.time.ZoneOffset;
import java.time.temporal.ChronoUnit;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashSet;
import org.junit.Before;
import org.junit.Rule;
import org.junit.Test;
import org.junit.rules.TestRule;
import org.mockito.Mockito;
@ -30,340 +24,27 @@
import io.debezium.connector.base.ChangeEventQueue;
import io.debezium.connector.oracle.junit.SkipTestDependingOnAdapterNameRule;
import io.debezium.connector.oracle.util.TestHelper;
import io.debezium.doc.FixFor;
import io.debezium.pipeline.DataChangeEvent;
import io.debezium.pipeline.source.spi.EventMetadataProvider;
/**
* Test streaming metrics.
* Common multi-adapter streaming metrics tests.
*/
public class OracleStreamingMetricsTest {
public abstract class OracleStreamingMetricsTest<T extends AbstractOracleStreamingChangeEventSourceMetrics> {
@Rule
public TestRule skipRule = new SkipTestDependingOnAdapterNameRule();
private OracleConnectorConfig connectorConfig;
private OracleStreamingChangeEventSourceMetrics metrics;
private Clock fixedClock;
protected OracleConnectorConfig connectorConfig;
protected T metrics;
protected Clock fixedClock;
@Before
public void before() {
init(TestHelper.defaultConfig());
}
@Test
public void testMetrics() {
metrics.setLastCapturedDmlCount(1);
assertThat(metrics.getTotalCapturedDmlCount() == 1).isTrue();
metrics.setCurrentScn(Scn.valueOf(1000L));
assertThat(metrics.getCurrentScn()).isEqualTo("1000");
metrics.setBatchSize(10);
assertThat(metrics.getBatchSize() == connectorConfig.getLogMiningBatchSizeDefault()).isTrue();
metrics.setBatchSize(1_000_000);
assertThat(metrics.getBatchSize()).isEqualTo(connectorConfig.getLogMiningBatchSizeDefault());
metrics.setBatchSize(6000);
assertThat(metrics.getBatchSize()).isEqualTo(6_000);
assertThat(metrics.getMillisecondToSleepBetweenMiningQuery()).isEqualTo(1000);
metrics.changeSleepingTime(true);
assertThat(metrics.getMillisecondToSleepBetweenMiningQuery()).isEqualTo(1200);
metrics.changeSleepingTime(false);
assertThat(metrics.getMillisecondToSleepBetweenMiningQuery()).isEqualTo(1000);
metrics.setMillisecondToSleepBetweenMiningQuery(-1L);
assertThat(metrics.getMillisecondToSleepBetweenMiningQuery()).isEqualTo(1000);
metrics.setMillisecondToSleepBetweenMiningQuery(4000L);
assertThat(metrics.getMillisecondToSleepBetweenMiningQuery()).isEqualTo(1000);
metrics.setMillisecondToSleepBetweenMiningQuery(2000L);
assertThat(metrics.getMillisecondToSleepBetweenMiningQuery()).isEqualTo(2000);
metrics.setLastDurationOfBatchCapturing(Duration.ofMillis(100));
assertThat(metrics.getLastDurationOfFetchQueryInMilliseconds()).isEqualTo(100);
metrics.setLastDurationOfBatchCapturing(Duration.ofMillis(200));
assertThat(metrics.getLastDurationOfFetchQueryInMilliseconds()).isEqualTo(200);
assertThat(metrics.getMaxDurationOfFetchQueryInMilliseconds()).isEqualTo(200);
assertThat(metrics.getFetchingQueryCount()).isEqualTo(2);
metrics.setCurrentLogFileName(new HashSet<>(Arrays.asList("name", "name1")));
assertThat(metrics.getCurrentRedoLogFileName()[0].equals("name")).isTrue();
assertThat(metrics.getCurrentRedoLogFileName()[1].equals("name1")).isTrue();
metrics.setSwitchCount(5);
assertThat(metrics.getSwitchCounter() == 5).isTrue();
metrics.reset();
metrics.setLastDurationOfBatchCapturing(Duration.ofMillis(1000));
assertThat(metrics.getLastDurationOfFetchQueryInMilliseconds()).isEqualTo(1000);
assertThat(metrics.getFetchingQueryCount()).isEqualTo(1);
metrics.reset();
metrics.setLastCapturedDmlCount(300);
metrics.setLastDurationOfBatchProcessing(Duration.ofMillis(1000));
assertThat(metrics.getLastCapturedDmlCount()).isEqualTo(300);
assertThat(metrics.getLastBatchProcessingTimeInMilliseconds()).isEqualTo(1000);
assertThat(metrics.getAverageBatchProcessingThroughput()).isGreaterThanOrEqualTo(300);
assertThat(metrics.getMaxCapturedDmlInBatch()).isEqualTo(300);
assertThat(metrics.getMaxBatchProcessingThroughput()).isEqualTo(300);
metrics.setLastCapturedDmlCount(500);
metrics.setLastDurationOfBatchProcessing(Duration.ofMillis(1000));
assertThat(metrics.getAverageBatchProcessingThroughput()).isEqualTo(400);
assertThat(metrics.getMaxCapturedDmlInBatch()).isEqualTo(500);
assertThat(metrics.getMaxBatchProcessingThroughput()).isEqualTo(500);
assertThat(metrics.getLastBatchProcessingThroughput()).isEqualTo(500);
metrics.setLastDurationOfBatchProcessing(Duration.ofMillis(5000));
assertThat(metrics.getLastBatchProcessingThroughput()).isEqualTo(100);
metrics.setLastDurationOfBatchProcessing(Duration.ZERO);
assertThat(metrics.getLastBatchProcessingThroughput()).isEqualTo(0);
assertThat(metrics.getHoursToKeepTransactionInBuffer()).isEqualTo(0);
assertThat(metrics.getMillisecondsToKeepTransactionsInBuffer()).isEqualTo(0L);
metrics.setRedoLogStatus(Collections.singletonMap("name", "current"));
assertThat(metrics.getRedoLogStatus()[0].equals("name | current")).isTrue();
assertThat(metrics.toString().contains("logMinerQueryCount"));
metrics.incrementNetworkConnectionProblemsCounter();
assertThat(metrics.getNetworkConnectionProblemsCounter()).isEqualTo(1);
metrics.setBatchSize(5000);
metrics.changeBatchSize(true, false);
assertThat(metrics.getBatchSize()).isEqualTo(6000);
metrics.changeBatchSize(false, false);
assertThat(metrics.getBatchSize()).isEqualTo(5000);
}
@Test
public void testLagMetrics() {
// no time difference between connector and database
long lag = metrics.getLagFromSourceInMilliseconds();
assertThat(lag).isEqualTo(0);
Instant dbEventTime = fixedClock.instant().minusMillis(2000);
metrics.calculateLagMetrics(dbEventTime);
lag = metrics.getLagFromSourceInMilliseconds();
assertThat(lag).isEqualTo(2000);
assertThat(metrics.getMaxLagFromSourceInMilliseconds()).isEqualTo(2000);
assertThat(metrics.getMinLagFromSourceInMilliseconds()).isEqualTo(2000);
// not realistic scenario
dbEventTime = fixedClock.instant().plusMillis(3000);
metrics.calculateLagMetrics(dbEventTime);
lag = metrics.getLagFromSourceInMilliseconds();
assertThat(lag).isEqualTo(3000);
assertThat(metrics.getMaxLagFromSourceInMilliseconds()).isEqualTo(3000);
assertThat(metrics.getMinLagFromSourceInMilliseconds()).isEqualTo(2000);
metrics.reset();
// ##########################
// the database time is ahead 1s and has an offset of +12h
OffsetDateTime dbTime = OffsetDateTime.parse("2021-05-16T00:30:01.00+12:00");
metrics.calculateTimeDifference(dbTime);
dbEventTime = Instant.parse("2021-05-16T00:29:58.00Z");
metrics.calculateLagMetrics(dbEventTime);
lag = metrics.getLagFromSourceInMilliseconds();
assertThat(lag).isEqualTo(3000);
assertThat(metrics.getMaxLagFromSourceInMilliseconds()).isEqualTo(3000);
assertThat(metrics.getMinLagFromSourceInMilliseconds()).isEqualTo(3000);
dbEventTime = Instant.parse("2021-05-16T00:29:57.00Z");
metrics.calculateLagMetrics(dbEventTime);
lag = metrics.getLagFromSourceInMilliseconds();
assertThat(lag).isEqualTo(4000);
assertThat(metrics.getMaxLagFromSourceInMilliseconds()).isEqualTo(4000);
assertThat(metrics.getMinLagFromSourceInMilliseconds()).isEqualTo(3000);
metrics.reset();
// ##########################
// the database time is ahead 1s and has an offset of +0h (UTC)
dbTime = OffsetDateTime.parse("2021-05-15T12:30:01.00Z");
metrics.calculateTimeDifference(dbTime);
dbEventTime = Instant.parse("2021-05-15T12:29:58.00Z");
metrics.calculateLagMetrics(dbEventTime);
lag = metrics.getLagFromSourceInMilliseconds();
assertThat(lag).isEqualTo(3000);
assertThat(metrics.getMaxLagFromSourceInMilliseconds()).isEqualTo(3000);
assertThat(metrics.getMinLagFromSourceInMilliseconds()).isEqualTo(3000);
dbEventTime = Instant.parse("2021-05-15T12:29:57.00Z");
metrics.calculateLagMetrics(dbEventTime);
lag = metrics.getLagFromSourceInMilliseconds();
assertThat(lag).isEqualTo(4000);
assertThat(metrics.getMaxLagFromSourceInMilliseconds()).isEqualTo(4000);
assertThat(metrics.getMinLagFromSourceInMilliseconds()).isEqualTo(3000);
metrics.reset();
// ##########################
// the database time is ahead 1s and has an offset of -12h
dbTime = OffsetDateTime.parse("2021-05-15T00:30:01.00-12:00");
metrics.calculateTimeDifference(dbTime);
dbEventTime = Instant.parse("2021-05-15T00:29:58.00Z");
metrics.calculateLagMetrics(dbEventTime);
lag = metrics.getLagFromSourceInMilliseconds();
assertThat(lag).isEqualTo(3000);
assertThat(metrics.getMaxLagFromSourceInMilliseconds()).isEqualTo(3000);
assertThat(metrics.getMinLagFromSourceInMilliseconds()).isEqualTo(3000);
dbEventTime = Instant.parse("2021-05-15T00:29:57.00Z");
metrics.calculateLagMetrics(dbEventTime);
lag = metrics.getLagFromSourceInMilliseconds();
assertThat(lag).isEqualTo(4000);
assertThat(metrics.getMaxLagFromSourceInMilliseconds()).isEqualTo(4000);
assertThat(metrics.getMinLagFromSourceInMilliseconds()).isEqualTo(3000);
metrics.reset();
// ##########################
// the database time is behind 1s and has an offset of +12h
dbTime = OffsetDateTime.parse("2021-05-16T00:29:59.00+12:00");
metrics.calculateTimeDifference(dbTime);
dbEventTime = Instant.parse("2021-05-16T00:29:58.00Z");
metrics.calculateLagMetrics(dbEventTime);
lag = metrics.getLagFromSourceInMilliseconds();
assertThat(lag).isEqualTo(1000);
assertThat(metrics.getMaxLagFromSourceInMilliseconds()).isEqualTo(1000);
assertThat(metrics.getMinLagFromSourceInMilliseconds()).isEqualTo(1000);
// ##########################
// the database time is behind 1s and has an offset of +0h (UTC)
dbTime = OffsetDateTime.parse("2021-05-15T12:29:59.00Z");
metrics.calculateTimeDifference(dbTime);
dbEventTime = Instant.parse("2021-05-15T12:29:58.00Z");
metrics.calculateLagMetrics(dbEventTime);
lag = metrics.getLagFromSourceInMilliseconds();
assertThat(lag).isEqualTo(1000);
assertThat(metrics.getMaxLagFromSourceInMilliseconds()).isEqualTo(1000);
assertThat(metrics.getMinLagFromSourceInMilliseconds()).isEqualTo(1000);
// ##########################
// the database time is behind 1s and has an offset of -12h
dbTime = OffsetDateTime.parse("2021-05-15T00:29:59.00-12:00");
metrics.calculateTimeDifference(dbTime);
dbEventTime = Instant.parse("2021-05-15T00:29:58.00Z");
metrics.calculateLagMetrics(dbEventTime);
lag = metrics.getLagFromSourceInMilliseconds();
assertThat(lag).isEqualTo(1000);
assertThat(metrics.getMaxLagFromSourceInMilliseconds()).isEqualTo(1000);
assertThat(metrics.getMinLagFromSourceInMilliseconds()).isEqualTo(1000);
}
@Test
public void testOtherMetrics() {
metrics.incrementScnFreezeCount();
assertThat(metrics.getScnFreezeCount()).isEqualTo(1);
metrics.incrementErrorCount();
assertThat(metrics.getErrorCount()).isEqualTo(1);
metrics.incrementWarningCount();
assertThat(metrics.getWarningCount()).isEqualTo(1);
metrics.incrementCommittedDmlCount(5_000);
for (int i = 0; i < 1000; i++) {
metrics.incrementRegisteredDmlCount();
metrics.incrementCommittedTransactions();
}
assertThat(metrics.getRegisteredDmlCount()).isEqualTo(1000);
assertThat(metrics.getNumberOfCommittedTransactions()).isEqualTo(1000);
assertThat(metrics.getCommitThroughput()).isGreaterThanOrEqualTo(1_000);
metrics.incrementOversizedTransactions();
assertThat(metrics.getNumberOfOversizedTransactions()).isEqualTo(1);
metrics.incrementRolledBackTransactions();
assertThat(metrics.getNumberOfRolledBackTransactions()).isEqualTo(1);
metrics.setActiveTransactions(5);
assertThat(metrics.getNumberOfActiveTransactions()).isEqualTo(5);
metrics.addRolledBackTransactionId("rolledback id");
assertThat(metrics.getNumberOfRolledBackTransactions()).isEqualTo(1);
assertThat(metrics.getRolledBackTransactionIds().contains("rolledback id")).isTrue();
metrics.addAbandonedTransactionId("abandoned id");
assertThat(metrics.getAbandonedTransactionIds().size()).isEqualTo(1);
assertThat(metrics.getAbandonedTransactionIds().contains("abandoned id")).isTrue();
metrics.setOldestScn(Scn.valueOf(10L));
assertThat(metrics.getOldestScn()).isEqualTo("10");
metrics.setCommittedScn(Scn.valueOf(10L));
assertThat(metrics.getCommittedScn()).isEqualTo("10");
assertThat(metrics.toString().contains("registeredDmlCount=1000")).isTrue();
metrics.setLastCommitDuration(Duration.ofMillis(100L));
assertThat(metrics.getLastCommitDurationInMilliseconds()).isEqualTo(100L);
metrics.setLastCommitDuration(Duration.ofMillis(50L));
assertThat(metrics.getMaxCommitDurationInMilliseconds()).isEqualTo(100L);
metrics.setOffsetScn(Scn.valueOf(10L));
assertThat(metrics.getOldestScn()).isEqualTo("10");
}
@Test
@FixFor("DBZ-2754")
public void testCustomTransactionRetention() throws Exception {
init(TestHelper.defaultConfig().with(OracleConnectorConfig.LOG_MINING_TRANSACTION_RETENTION, 3));
assertThat(metrics.getHoursToKeepTransactionInBuffer()).isEqualTo(3);
assertThat(metrics.getMillisecondsToKeepTransactionsInBuffer()).isEqualTo(3 * 3600000);
}
@Test
@FixFor("DBZ-5179")
public void testRollbackTransactionIdSetSizeLimit() throws Exception {
init(TestHelper.defaultConfig().with(OracleConnectorConfig.LOG_MINING_TRANSACTION_RETENTION, 3));
// Check state up to maximum size
for (int i = 1; i <= 10; ++i) {
metrics.addRolledBackTransactionId(String.valueOf(i));
}
assertThat(metrics.getRolledBackTransactionIds()).containsOnly("1", "2", "3", "4", "5", "6", "7", "8", "9", "10");
// Add another rollback transaction, does not exist in set
metrics.addRolledBackTransactionId("11");
assertThat(metrics.getRolledBackTransactionIds()).containsOnly("2", "3", "4", "5", "6", "7", "8", "9", "10", "11");
// Add another rollback transaction, this time the same as before
// Set should be unchanged.
metrics.addRolledBackTransactionId("11");
assertThat(metrics.getRolledBackTransactionIds()).containsOnly("2", "3", "4", "5", "6", "7", "8", "9", "10", "11");
}
@Test
@FixFor("DBZ-5179")
public void testAbandonedTransactionIdSetSizeLimit() throws Exception {
init(TestHelper.defaultConfig().with(OracleConnectorConfig.LOG_MINING_TRANSACTION_RETENTION, 3));
// Check state up to maximum size
for (int i = 1; i <= 10; ++i) {
metrics.addAbandonedTransactionId(String.valueOf(i));
}
assertThat(metrics.getAbandonedTransactionIds()).containsOnly("1", "2", "3", "4", "5", "6", "7", "8", "9", "10");
// Add another abandoned transaction, does not exist in set
metrics.addAbandonedTransactionId("11");
assertThat(metrics.getAbandonedTransactionIds()).containsOnly("2", "3", "4", "5", "6", "7", "8", "9", "10", "11");
// Add another abandoned transaction, this time the same as before
// Set should be unchanged.
metrics.addAbandonedTransactionId("11");
assertThat(metrics.getAbandonedTransactionIds()).containsOnly("2", "3", "4", "5", "6", "7", "8", "9", "10", "11");
}
private void init(Configuration.Builder builder) {
protected void init(Configuration.Builder builder) {
this.connectorConfig = new OracleConnectorConfig(builder.build());
final ChangeEventQueue<DataChangeEvent> queue = new ChangeEventQueue.Builder<DataChangeEvent>()
@ -378,6 +59,13 @@ private void init(Configuration.Builder builder) {
final OracleEventMetadataProvider metadataProvider = new OracleEventMetadataProvider();
fixedClock = Clock.fixed(Instant.parse("2021-05-15T12:30:00.00Z"), ZoneOffset.UTC);
this.metrics = new OracleStreamingChangeEventSourceMetrics(taskContext, queue, metadataProvider, connectorConfig, fixedClock);
this.metrics = createMetrics(taskContext, queue, metadataProvider, connectorConfig, fixedClock);
}
protected abstract T createMetrics(OracleTaskContext taskContext,
ChangeEventQueue<DataChangeEvent> queue,
EventMetadataProvider metadataProvider,
OracleConnectorConfig connectorConfig,
Clock clock);
}

View File

@ -0,0 +1,330 @@
/*
* Copyright Debezium Authors.
*
* Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0
*/
package io.debezium.connector.oracle.logminer;
import static org.assertj.core.api.Assertions.assertThat;
import java.time.Clock;
import java.time.Duration;
import java.time.Instant;
import java.time.OffsetDateTime;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashSet;
import org.junit.Test;
import io.debezium.connector.base.ChangeEventQueue;
import io.debezium.connector.oracle.OracleConnectorConfig;
import io.debezium.connector.oracle.OracleStreamingMetricsTest;
import io.debezium.connector.oracle.OracleTaskContext;
import io.debezium.connector.oracle.Scn;
import io.debezium.connector.oracle.junit.SkipWhenAdapterNameIsNot;
import io.debezium.connector.oracle.util.TestHelper;
import io.debezium.doc.FixFor;
import io.debezium.pipeline.DataChangeEvent;
import io.debezium.pipeline.source.spi.EventMetadataProvider;
/**
* @author Chris Cranford
*/
@SkipWhenAdapterNameIsNot(SkipWhenAdapterNameIsNot.AdapterName.LOGMINER)
public class LogMinerStreamMetricsTest extends OracleStreamingMetricsTest<LogMinerStreamingChangeEventSourceMetrics> {
@Override
protected LogMinerStreamingChangeEventSourceMetrics createMetrics(OracleTaskContext taskContext,
ChangeEventQueue<DataChangeEvent> queue,
EventMetadataProvider metadataProvider,
OracleConnectorConfig connectorConfig,
Clock clock) {
return new LogMinerStreamingChangeEventSourceMetrics(taskContext, queue, metadataProvider, connectorConfig, clock);
}
@Test
public void testMetrics() {
metrics.setLastCapturedDmlCount(1);
assertThat(metrics.getTotalCapturedDmlCount() == 1).isTrue();
metrics.setCurrentScn(Scn.valueOf(1000L));
assertThat(metrics.getCurrentScn()).isEqualTo("1000");
metrics.setLastDurationOfFetchQuery(Duration.ofMillis(100));
assertThat(metrics.getLastDurationOfFetchQueryInMilliseconds()).isEqualTo(100);
metrics.setLastDurationOfFetchQuery(Duration.ofMillis(200));
assertThat(metrics.getLastDurationOfFetchQueryInMilliseconds()).isEqualTo(200);
assertThat(metrics.getMaxDurationOfFetchQueryInMilliseconds()).isEqualTo(200);
assertThat(metrics.getFetchingQueryCount()).isEqualTo(2);
metrics.setCurrentLogFileNames(new HashSet<>(Arrays.asList("name", "name1")));
assertThat(metrics.getCurrentRedoLogFileName()[0].equals("name")).isTrue();
assertThat(metrics.getCurrentRedoLogFileName()[1].equals("name1")).isTrue();
metrics.setSwitchCount(5);
assertThat(metrics.getSwitchCounter() == 5).isTrue();
metrics.reset();
metrics.setLastDurationOfFetchQuery(Duration.ofMillis(1000));
assertThat(metrics.getLastDurationOfFetchQueryInMilliseconds()).isEqualTo(1000);
assertThat(metrics.getFetchingQueryCount()).isEqualTo(1);
metrics.reset();
metrics.setLastCapturedDmlCount(300);
metrics.setLastBatchProcessingDuration(Duration.ofMillis(1000));
assertThat(metrics.getLastCapturedDmlCount()).isEqualTo(300);
assertThat(metrics.getLastBatchProcessingTimeInMilliseconds()).isEqualTo(1000);
assertThat(metrics.getAverageBatchProcessingThroughput()).isGreaterThanOrEqualTo(300);
assertThat(metrics.getMaxCapturedDmlInBatch()).isEqualTo(300);
assertThat(metrics.getMaxBatchProcessingThroughput()).isEqualTo(300);
metrics.setLastCapturedDmlCount(500);
metrics.setLastBatchProcessingDuration(Duration.ofMillis(1000));
assertThat(metrics.getAverageBatchProcessingThroughput()).isEqualTo(400);
assertThat(metrics.getMaxCapturedDmlInBatch()).isEqualTo(500);
assertThat(metrics.getMaxBatchProcessingThroughput()).isEqualTo(500);
assertThat(metrics.getLastBatchProcessingThroughput()).isEqualTo(500);
metrics.setLastBatchProcessingDuration(Duration.ofMillis(5000));
assertThat(metrics.getLastBatchProcessingThroughput()).isEqualTo(100);
metrics.setLastBatchProcessingDuration(Duration.ZERO);
assertThat(metrics.getLastBatchProcessingThroughput()).isEqualTo(0);
assertThat(metrics.getHoursToKeepTransactionInBuffer()).isEqualTo(0);
assertThat(metrics.getMillisecondsToKeepTransactionsInBuffer()).isEqualTo(0L);
metrics.setRedoLogStatuses(Collections.singletonMap("name", "current"));
assertThat(metrics.getRedoLogStatus()[0].equals("name | current")).isTrue();
assertThat(metrics.toString().contains("logMinerQueryCount"));
}
@Test
public void testLagMetrics() {
// no time difference between connector and database
long lag = metrics.getLagFromSourceInMilliseconds();
assertThat(lag).isEqualTo(0);
Instant dbEventTime = fixedClock.instant().minusMillis(2000);
metrics.calculateLagFromSource(dbEventTime);
lag = metrics.getLagFromSourceInMilliseconds();
assertThat(lag).isEqualTo(2000);
assertThat(metrics.getMaxLagFromSourceInMilliseconds()).isEqualTo(2000);
assertThat(metrics.getMinLagFromSourceInMilliseconds()).isEqualTo(2000);
// not realistic scenario
dbEventTime = fixedClock.instant().plusMillis(3000);
metrics.calculateLagFromSource(dbEventTime);
lag = metrics.getLagFromSourceInMilliseconds();
assertThat(lag).isEqualTo(3000);
assertThat(metrics.getMaxLagFromSourceInMilliseconds()).isEqualTo(3000);
assertThat(metrics.getMinLagFromSourceInMilliseconds()).isEqualTo(2000);
metrics.reset();
// ##########################
// the database time is ahead 1s and has an offset of +12h
OffsetDateTime dbTime = OffsetDateTime.parse("2021-05-16T00:30:01.00+12:00");
metrics.setDatabaseTimeDifference(dbTime);
dbEventTime = Instant.parse("2021-05-16T00:29:58.00Z");
metrics.calculateLagFromSource(dbEventTime);
lag = metrics.getLagFromSourceInMilliseconds();
assertThat(lag).isEqualTo(3000);
assertThat(metrics.getMaxLagFromSourceInMilliseconds()).isEqualTo(3000);
assertThat(metrics.getMinLagFromSourceInMilliseconds()).isEqualTo(3000);
dbEventTime = Instant.parse("2021-05-16T00:29:57.00Z");
metrics.calculateLagFromSource(dbEventTime);
lag = metrics.getLagFromSourceInMilliseconds();
assertThat(lag).isEqualTo(4000);
assertThat(metrics.getMaxLagFromSourceInMilliseconds()).isEqualTo(4000);
assertThat(metrics.getMinLagFromSourceInMilliseconds()).isEqualTo(3000);
metrics.reset();
// ##########################
// the database time is ahead 1s and has an offset of +0h (UTC)
dbTime = OffsetDateTime.parse("2021-05-15T12:30:01.00Z");
metrics.setDatabaseTimeDifference(dbTime);
dbEventTime = Instant.parse("2021-05-15T12:29:58.00Z");
metrics.calculateLagFromSource(dbEventTime);
lag = metrics.getLagFromSourceInMilliseconds();
assertThat(lag).isEqualTo(3000);
assertThat(metrics.getMaxLagFromSourceInMilliseconds()).isEqualTo(3000);
assertThat(metrics.getMinLagFromSourceInMilliseconds()).isEqualTo(3000);
dbEventTime = Instant.parse("2021-05-15T12:29:57.00Z");
metrics.calculateLagFromSource(dbEventTime);
lag = metrics.getLagFromSourceInMilliseconds();
assertThat(lag).isEqualTo(4000);
assertThat(metrics.getMaxLagFromSourceInMilliseconds()).isEqualTo(4000);
assertThat(metrics.getMinLagFromSourceInMilliseconds()).isEqualTo(3000);
metrics.reset();
// ##########################
// the database time is ahead 1s and has an offset of -12h
dbTime = OffsetDateTime.parse("2021-05-15T00:30:01.00-12:00");
metrics.setDatabaseTimeDifference(dbTime);
dbEventTime = Instant.parse("2021-05-15T00:29:58.00Z");
metrics.calculateLagFromSource(dbEventTime);
lag = metrics.getLagFromSourceInMilliseconds();
assertThat(lag).isEqualTo(3000);
assertThat(metrics.getMaxLagFromSourceInMilliseconds()).isEqualTo(3000);
assertThat(metrics.getMinLagFromSourceInMilliseconds()).isEqualTo(3000);
dbEventTime = Instant.parse("2021-05-15T00:29:57.00Z");
metrics.calculateLagFromSource(dbEventTime);
lag = metrics.getLagFromSourceInMilliseconds();
assertThat(lag).isEqualTo(4000);
assertThat(metrics.getMaxLagFromSourceInMilliseconds()).isEqualTo(4000);
assertThat(metrics.getMinLagFromSourceInMilliseconds()).isEqualTo(3000);
metrics.reset();
// ##########################
// the database time is behind 1s and has an offset of +12h
dbTime = OffsetDateTime.parse("2021-05-16T00:29:59.00+12:00");
metrics.setDatabaseTimeDifference(dbTime);
dbEventTime = Instant.parse("2021-05-16T00:29:58.00Z");
metrics.calculateLagFromSource(dbEventTime);
lag = metrics.getLagFromSourceInMilliseconds();
assertThat(lag).isEqualTo(1000);
assertThat(metrics.getMaxLagFromSourceInMilliseconds()).isEqualTo(1000);
assertThat(metrics.getMinLagFromSourceInMilliseconds()).isEqualTo(1000);
// ##########################
// the database time is behind 1s and has an offset of +0h (UTC)
dbTime = OffsetDateTime.parse("2021-05-15T12:29:59.00Z");
metrics.setDatabaseTimeDifference(dbTime);
dbEventTime = Instant.parse("2021-05-15T12:29:58.00Z");
metrics.calculateLagFromSource(dbEventTime);
lag = metrics.getLagFromSourceInMilliseconds();
assertThat(lag).isEqualTo(1000);
assertThat(metrics.getMaxLagFromSourceInMilliseconds()).isEqualTo(1000);
assertThat(metrics.getMinLagFromSourceInMilliseconds()).isEqualTo(1000);
// ##########################
// the database time is behind 1s and has an offset of -12h
dbTime = OffsetDateTime.parse("2021-05-15T00:29:59.00-12:00");
metrics.setDatabaseTimeDifference(dbTime);
dbEventTime = Instant.parse("2021-05-15T00:29:58.00Z");
metrics.calculateLagFromSource(dbEventTime);
lag = metrics.getLagFromSourceInMilliseconds();
assertThat(lag).isEqualTo(1000);
assertThat(metrics.getMaxLagFromSourceInMilliseconds()).isEqualTo(1000);
assertThat(metrics.getMinLagFromSourceInMilliseconds()).isEqualTo(1000);
}
@Test
public void testOtherMetrics() {
metrics.incrementScnFreezeCount();
assertThat(metrics.getScnFreezeCount()).isEqualTo(1);
metrics.incrementErrorCount();
assertThat(metrics.getErrorCount()).isEqualTo(1);
metrics.incrementWarningCount();
assertThat(metrics.getWarningCount()).isEqualTo(1);
for (int i = 0; i < 1000; i++) {
metrics.incrementTotalChangesCount();
metrics.incrementCommittedTransactionCount();
}
assertThat(metrics.getRegisteredDmlCount()).isEqualTo(1000);
assertThat(metrics.getNumberOfCommittedTransactions()).isEqualTo(1000);
assertThat(metrics.getCommitThroughput()).isGreaterThanOrEqualTo(1_000);
metrics.incrementOversizedTransactionCount();
assertThat(metrics.getNumberOfOversizedTransactions()).isEqualTo(1);
metrics.incrementRolledBackTransactionCount();
assertThat(metrics.getNumberOfRolledBackTransactions()).isEqualTo(1);
metrics.setActiveTransactionCount(5);
assertThat(metrics.getNumberOfActiveTransactions()).isEqualTo(5);
metrics.addRolledBackTransactionId("rolledback id");
assertThat(metrics.getNumberOfRolledBackTransactions()).isEqualTo(1);
assertThat(metrics.getRolledBackTransactionIds().contains("rolledback id")).isTrue();
metrics.addAbandonedTransactionId("abandoned id");
assertThat(metrics.getAbandonedTransactionIds().size()).isEqualTo(1);
assertThat(metrics.getAbandonedTransactionIds().contains("abandoned id")).isTrue();
metrics.setOldestScnDetails(Scn.valueOf(10L), null);
assertThat(metrics.getOldestScn()).isEqualTo("10");
metrics.setCommitScn(Scn.valueOf(10L));
assertThat(metrics.getCommittedScn()).isEqualTo("10");
assertThat(metrics.toString().contains("changesCount=1000")).isTrue();
metrics.setLastCommitDuration(Duration.ofMillis(100L));
assertThat(metrics.getLastCommitDurationInMilliseconds()).isEqualTo(100L);
metrics.setLastCommitDuration(Duration.ofMillis(50L));
assertThat(metrics.getMaxCommitDurationInMilliseconds()).isEqualTo(100L);
metrics.setOffsetScn(Scn.valueOf(10L));
assertThat(metrics.getOldestScn()).isEqualTo("10");
}
@Test
@FixFor("DBZ-2754")
public void testCustomTransactionRetention() throws Exception {
init(TestHelper.defaultConfig().with(OracleConnectorConfig.LOG_MINING_TRANSACTION_RETENTION, 3));
assertThat(metrics.getHoursToKeepTransactionInBuffer()).isEqualTo(3);
assertThat(metrics.getMillisecondsToKeepTransactionsInBuffer()).isEqualTo(3 * 3600000);
}
@Test
@FixFor("DBZ-5179")
public void testRollbackTransactionIdSetSizeLimit() throws Exception {
init(TestHelper.defaultConfig().with(OracleConnectorConfig.LOG_MINING_TRANSACTION_RETENTION, 3));
// Check state up to maximum size
for (int i = 1; i <= 10; ++i) {
metrics.addRolledBackTransactionId(String.valueOf(i));
}
assertThat(metrics.getRolledBackTransactionIds()).containsOnly("1", "2", "3", "4", "5", "6", "7", "8", "9", "10");
// Add another rollback transaction, does not exist in set
metrics.addRolledBackTransactionId("11");
assertThat(metrics.getRolledBackTransactionIds()).containsOnly("2", "3", "4", "5", "6", "7", "8", "9", "10", "11");
// Add another rollback transaction, this time the same as before
// Set should be unchanged.
metrics.addRolledBackTransactionId("11");
assertThat(metrics.getRolledBackTransactionIds()).containsOnly("2", "3", "4", "5", "6", "7", "8", "9", "10", "11");
}
@Test
@FixFor("DBZ-5179")
public void testAbandonedTransactionIdSetSizeLimit() throws Exception {
init(TestHelper.defaultConfig().with(OracleConnectorConfig.LOG_MINING_TRANSACTION_RETENTION, 3));
// Check state up to maximum size
for (int i = 1; i <= 10; ++i) {
metrics.addAbandonedTransactionId(String.valueOf(i));
}
assertThat(metrics.getAbandonedTransactionIds()).containsOnly("1", "2", "3", "4", "5", "6", "7", "8", "9", "10");
// Add another abandoned transaction, does not exist in set
metrics.addAbandonedTransactionId("11");
assertThat(metrics.getAbandonedTransactionIds()).containsOnly("2", "3", "4", "5", "6", "7", "8", "9", "10", "11");
// Add another abandoned transaction, this time the same as before
// Set should be unchanged.
metrics.addAbandonedTransactionId("11");
assertThat(metrics.getAbandonedTransactionIds()).containsOnly("2", "3", "4", "5", "6", "7", "8", "9", "10", "11");
}
}

View File

@ -36,12 +36,12 @@
import io.debezium.connector.oracle.OracleDefaultValueConverter;
import io.debezium.connector.oracle.OracleOffsetContext;
import io.debezium.connector.oracle.OraclePartition;
import io.debezium.connector.oracle.OracleStreamingChangeEventSourceMetrics;
import io.debezium.connector.oracle.OracleTaskContext;
import io.debezium.connector.oracle.OracleValueConverters;
import io.debezium.connector.oracle.Scn;
import io.debezium.connector.oracle.StreamingAdapter.TableNameCaseSensitivity;
import io.debezium.connector.oracle.junit.SkipTestDependingOnAdapterNameRule;
import io.debezium.connector.oracle.logminer.LogMinerStreamingChangeEventSourceMetrics;
import io.debezium.connector.oracle.logminer.events.EventType;
import io.debezium.connector.oracle.logminer.events.LogMinerEventRow;
import io.debezium.connector.oracle.util.TestHelper;
@ -74,7 +74,7 @@ public abstract class AbstractProcessorUnitTest<T extends AbstractLogMinerEventP
protected ChangeEventSourceContext context;
protected EventDispatcher<OraclePartition, TableId> dispatcher;
protected OracleDatabaseSchema schema;
protected OracleStreamingChangeEventSourceMetrics metrics;
protected LogMinerStreamingChangeEventSourceMetrics metrics;
protected OraclePartition partition;
protected OracleOffsetContext offsetContext;
protected OracleConnection connection;
@ -361,7 +361,7 @@ private OracleConnection createOracleConnection(boolean singleOptionalValueThrow
return connection;
}
private OracleStreamingChangeEventSourceMetrics createMetrics(OracleDatabaseSchema schema) throws Exception {
private LogMinerStreamingChangeEventSourceMetrics createMetrics(OracleDatabaseSchema schema) throws Exception {
final OracleConnectorConfig connectorConfig = new OracleConnectorConfig(getConfig().build());
final OracleTaskContext taskContext = new OracleTaskContext(connectorConfig, schema);
@ -371,7 +371,7 @@ private OracleStreamingChangeEventSourceMetrics createMetrics(OracleDatabaseSche
.maxQueueSize(DEFAULT_MAX_QUEUE_SIZE)
.build();
return new OracleStreamingChangeEventSourceMetrics(taskContext, queue, null, connectorConfig);
return new LogMinerStreamingChangeEventSourceMetrics(taskContext, queue, null, connectorConfig);
}
private LogMinerEventRow getStartLogMinerEventRow(Scn scn, String transactionId) {

View File

@ -0,0 +1,47 @@
/*
* Copyright Debezium Authors.
*
* Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0
*/
package io.debezium.connector.oracle.olr;
import static org.assertj.core.api.Assertions.assertThat;
import java.time.Clock;
import org.junit.Test;
import io.debezium.connector.base.ChangeEventQueue;
import io.debezium.connector.oracle.OracleConnectorConfig;
import io.debezium.connector.oracle.OracleStreamingMetricsTest;
import io.debezium.connector.oracle.OracleTaskContext;
import io.debezium.connector.oracle.Scn;
import io.debezium.connector.oracle.junit.SkipWhenAdapterNameIsNot;
import io.debezium.pipeline.DataChangeEvent;
import io.debezium.pipeline.source.spi.EventMetadataProvider;
/**
* OpenLogReplicator Streaming Metrics Tests
*
* @author Chris Cranford
*/
@SkipWhenAdapterNameIsNot(SkipWhenAdapterNameIsNot.AdapterName.OLR)
public class OpenLogReplicatorStreamingMetricsTest extends OracleStreamingMetricsTest<OpenLogReplicatorStreamingChangeEventSourceMetrics> {
@Override
protected OpenLogReplicatorStreamingChangeEventSourceMetrics createMetrics(OracleTaskContext taskContext,
ChangeEventQueue<DataChangeEvent> queue,
EventMetadataProvider metadataProvider,
OracleConnectorConfig connectorConfig,
Clock clock) {
return new OpenLogReplicatorStreamingChangeEventSourceMetrics(taskContext, queue, metadataProvider);
}
@Test
public void testCheckpointDetalisMetrics() {
metrics.setCheckpointDetails(Scn.valueOf("12345"), 98765L);
assertThat(metrics.getCheckpointScn()).isEqualTo(Scn.valueOf("12345").asBigInteger());
assertThat(metrics.getCheckpointIndex()).isEqualTo(98765L);
}
}

View File

@ -0,0 +1,35 @@
/*
* Copyright Debezium Authors.
*
* Licensed under the Apache Software License version 2.0, available at http://www.apache.org/licenses/LICENSE-2.0
*/
package io.debezium.connector.oracle.xstream;
import java.time.Clock;
import io.debezium.connector.base.ChangeEventQueue;
import io.debezium.connector.oracle.OracleConnectorConfig;
import io.debezium.connector.oracle.OracleStreamingMetricsTest;
import io.debezium.connector.oracle.OracleTaskContext;
import io.debezium.connector.oracle.junit.SkipWhenAdapterNameIsNot;
import io.debezium.pipeline.DataChangeEvent;
import io.debezium.pipeline.source.spi.EventMetadataProvider;
/**
* Oracle XStream Streaming Metrics Tests
*
* @author Chris Cranford
*/
@SkipWhenAdapterNameIsNot(SkipWhenAdapterNameIsNot.AdapterName.XSTREAM)
public class XStreamStreamingMetricsTest extends OracleStreamingMetricsTest<XStreamStreamingChangeEventSourceMetrics> {
@Override
protected XStreamStreamingChangeEventSourceMetrics createMetrics(OracleTaskContext taskContext,
ChangeEventQueue<DataChangeEvent> queue,
EventMetadataProvider metadataProvider,
OracleConnectorConfig connectorConfig,
Clock clock) {
return new XStreamStreamingChangeEventSourceMetrics(taskContext, queue, metadataProvider);
}
}