This is an automated email from the ASF dual-hosted git repository. irakov pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/ignite.git
The following commit(s) were added to refs/heads/master by this push: new fb3129f IGNITE-12063 Add ability to track system/user time held in transaction - Fixes #6772. fb3129f is described below commit fb3129f7501786ec07d82d9b6c85016a54029ad4 Author: denis-chudov <regmoongl...@gmail.com> AuthorDate: Thu Sep 5 20:04:11 2019 +0300 IGNITE-12063 Add ability to track system/user time held in transaction - Fixes #6772. Signed-off-by: Ivan Rakov <ira...@apache.org> --- .../org/apache/ignite/IgniteSystemProperties.java | 24 ++ .../org/apache/ignite/internal/IgniteFeatures.java | 7 +- .../ignite/internal/TransactionsMXBeanImpl.java | 30 ++ .../processors/cache/GridCacheGateway.java | 6 + .../cache/GridCachePartitionExchangeManager.java | 40 ++- .../processors/cache/GridCacheProcessor.java | 67 +++++ .../LongRunningTxTimeDumpSettingsClosure.java | 74 +++++ .../cache/distributed/near/GridNearTxLocal.java | 202 ++++++++++++- .../cache/transactions/IgniteTxManager.java | 163 +++++++++- .../transactions/TransactionMetricsAdapter.java | 64 ++++ .../cache/transactions/TransactionProxyImpl.java | 4 + .../processors/metric/impl/HistogramMetric.java | 11 + .../apache/ignite/mxbean/TransactionsMXBean.java | 90 ++++++ .../internal/metric/JmxMetricExporterSpiTest.java | 28 +- .../GridTransactionsSystemUserTimeMetricsTest.java | 327 +++++++++++++++++++++ .../TxDataConsistencyOnCommitFailureTest.java | 9 +- .../testframework/junits/GridAbstractTest.java | 31 ++ .../junits/common/GridCommonAbstractTest.java | 32 ++ .../ignite/testsuites/IgniteCacheTestSuite7.java | 3 + 19 files changed, 1180 insertions(+), 32 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java index 746c7a2..ad394e8 100644 --- a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java +++ b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java @@ -1253,6 +1253,30 @@ public final class IgniteSystemProperties { public static final String INDEX_REBUILDING_PARALLELISM = "INDEX_REBUILDING_PARALLELISM"; /** + * Threshold timeout for long transactions, if transaction exceeds it, it will be dumped in log with + * information about how much time did it spent in system time (time while aquiring locks, preparing, + * commiting, etc) and user time (time when client node runs some code while holding transaction and not + * waiting it). Equals 0 if not set. No long transactions are dumped in log if nor this parameter + * neither {@link #IGNITE_TRANSACTION_TIME_DUMP_SAMPLES_COEFFICIENT} is set. + */ + public static final String IGNITE_LONG_TRANSACTION_TIME_DUMP_THRESHOLD = "IGNITE_LONG_TRANSACTION_TIME_DUMP_THRESHOLD"; + + /** + * The coefficient for samples of completed transactions that will be dumped in log. Must be float value + * between 0.0 and 1.0 inclusive. Default value is <code>0.0</code>. + */ + public static final String IGNITE_TRANSACTION_TIME_DUMP_SAMPLES_COEFFICIENT = + "IGNITE_TRANSACTION_TIME_DUMP_SAMPLES_COEFFICIENT"; + + /** + * The limit of samples of completed transactions that will be dumped in log per second, if + * {@link #IGNITE_TRANSACTION_TIME_DUMP_SAMPLES_COEFFICIENT} is above <code>0.0</code>. Must be integer value + * greater than <code>0</code>. Default value is <code>5</code>. + */ + public static final String IGNITE_TRANSACTION_TIME_DUMP_SAMPLES_PER_SECOND_LIMIT = + "IGNITE_TRANSACTION_TIME_DUMP_SAMPLES_PER_SECOND_LIMIT"; + + /** * Enforces singleton. */ private IgniteSystemProperties() { diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteFeatures.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteFeatures.java index f356a9b..fad20f1 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/IgniteFeatures.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteFeatures.java @@ -56,7 +56,7 @@ public enum IgniteFeatures { /** Command which allow to detect and cleanup garbage which could left after destroying caches in shared groups */ FIND_AND_DELETE_GARBAGE_COMMAND(8), - + /** Support of cluster read-only mode. */ CLUSTER_READ_ONLY_MODE(9), @@ -70,7 +70,10 @@ public enum IgniteFeatures { CHANNEL_COMMUNICATION(12), /** Replacing TcpDiscoveryNode field with nodeId field in discovery messages. */ - TCP_DISCOVERY_MESSAGE_NODE_COMPACT_REPRESENTATION(14); + TCP_DISCOVERY_MESSAGE_NODE_COMPACT_REPRESENTATION(14), + + /** LRT system and user time dump settings. */ + LRT_SYSTEM_USER_TIME_DUMP_SETTINGS(18); /** * Unique feature identifier. diff --git a/modules/core/src/main/java/org/apache/ignite/internal/TransactionsMXBeanImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/TransactionsMXBeanImpl.java index 1969d29..f430439 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/TransactionsMXBeanImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/TransactionsMXBeanImpl.java @@ -142,6 +142,36 @@ public class TransactionsMXBeanImpl implements TransactionsMXBean { } /** {@inheritDoc} */ + @Override public long getLongTransactionTimeDumpThreshold() { + return ctx.cache().context().tm().longTransactionTimeDumpThreshold(); + } + + /** {@inheritDoc} */ + @Override public void setLongTransactionTimeDumpThreshold(long threshold) { + ctx.cache().longTransactionTimeDumpThreshold(threshold); + } + + /** {@inheritDoc} */ + @Override public double getTransactionTimeDumpSamplesCoefficient() { + return ctx.cache().context().tm().transactionTimeDumpSamplesCoefficient(); + } + + /** {@inheritDoc} */ + @Override public void setTransactionTimeDumpSamplesCoefficient(double coefficient) { + ctx.cache().transactionTimeDumpSamplesCoefficient(coefficient); + } + + /** {@inheritDoc} */ + @Override public int getTransactionTimeDumpSamplesPerSecondLimit() { + return ctx.cache().context().tm().transactionTimeDumpSamplesPerSecondLimit(); + } + + /** {@inheritDoc} */ + @Override public void setTransactionTimeDumpSamplesPerSecondLimit(int limit) { + ctx.cache().longTransactionTimeDumpSamplesPerSecondLimit(limit); + } + + /** {@inheritDoc} */ @Override public String toString() { return S.toString(TransactionsMXBeanImpl.class, this); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheGateway.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheGateway.java index 9a8ce7a..c068290 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheGateway.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheGateway.java @@ -133,6 +133,8 @@ public class GridCacheGateway<K, V> { ctx.tm().resetContext(); ctx.mvcc().contextReset(); + ctx.tm().leaveNearTxSystemSection(); + // Unwind eviction notifications. if (!ctx.shared().closed(ctx)) CU.unwindEvicts(ctx); @@ -170,6 +172,8 @@ public class GridCacheGateway<K, V> { ctx.name() + "]", e); } + ctx.tm().enterNearTxSystemSection(); + onEnter(opCtx); Lock lock = rwLock.readLock(); @@ -239,6 +243,8 @@ public class GridCacheGateway<K, V> { // Unwind eviction notifications. CU.unwindEvicts(ctx); + ctx.tm().leaveNearTxSystemSection(); + // Return back previous thread local operation context per call. ctx.operationContextPerCall(prev); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java index dc6923a..19299a4 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java @@ -100,6 +100,7 @@ import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.Sto import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.latch.ExchangeLatchManager; import org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridClientPartitionTopology; import org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtPartitionTopology; +import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxLocal; import org.apache.ignite.internal.processors.cache.persistence.snapshot.IgniteCacheSnapshotManager; import org.apache.ignite.internal.processors.cache.persistence.snapshot.SnapshotDiscoveryMessage; import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx; @@ -113,6 +114,7 @@ import org.apache.ignite.internal.processors.query.schema.SchemaNodeLeaveExchang import org.apache.ignite.internal.processors.timeout.GridTimeoutObject; import org.apache.ignite.internal.util.GridListSet; import org.apache.ignite.internal.util.GridPartitionStateMap; +import org.apache.ignite.internal.util.GridStringBuilder; import org.apache.ignite.internal.util.IgniteUtils; import org.apache.ignite.internal.util.future.GridCompoundFuture; import org.apache.ignite.internal.util.future.GridFinishedFuture; @@ -2030,6 +2032,41 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana } /** + * Builds warning string for long running transaction. + * + * @param tx Transaction. + * @param curTime Current timestamp. + * @return Warning string. + */ + private String longRunningTransactionWarning(IgniteInternalTx tx, long curTime) { + GridStringBuilder warning = new GridStringBuilder() + .a(">>> Transaction [startTime=") + .a(formatTime(tx.startTime())) + .a(", curTime=") + .a(formatTime(curTime)); + + if (tx instanceof GridNearTxLocal) { + GridNearTxLocal nearTxLoc = (GridNearTxLocal)tx; + + long sysTimeCurr = nearTxLoc.systemTimeCurrent(); + + //in some cases totalTimeMillis can be less than systemTimeMillis, as they are calculated with different precision + long userTime = Math.max(curTime - nearTxLoc.startTime() - sysTimeCurr, 0); + + warning.a(", systemTime=") + .a(sysTimeCurr) + .a(", userTime=") + .a(userTime); + } + + warning.a(", tx=") + .a(tx) + .a("]"); + + return warning.toString(); + } + + /** * @param timeout Operation timeout. * @return {@code True} if found long running operations. */ @@ -2055,8 +2092,7 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana found = true; if (warnings.canAddMessage()) { - warnings.add(">>> Transaction [startTime=" + formatTime(tx.startTime()) + - ", curTime=" + formatTime(curTime) + ", tx=" + tx + ']'); + warnings.add(longRunningTransactionWarning(tx, curTime)); if (ltrDumpLimiter.allowAction(tx)) dumpLongRunningTransaction(tx); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java index dfc74bb..da601fe 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java @@ -158,6 +158,7 @@ import org.apache.ignite.lang.IgniteClosure; import org.apache.ignite.lang.IgniteFuture; import org.apache.ignite.lang.IgniteInClosure; import org.apache.ignite.lang.IgnitePredicate; +import org.apache.ignite.lang.IgniteRunnable; import org.apache.ignite.lang.IgniteUuid; import org.apache.ignite.lifecycle.LifecycleAware; import org.apache.ignite.marshaller.Marshaller; @@ -187,6 +188,7 @@ import static org.apache.ignite.configuration.DeploymentMode.CONTINUOUS; import static org.apache.ignite.configuration.DeploymentMode.SHARED; import static org.apache.ignite.internal.GridComponent.DiscoveryDataExchangeType.CACHE_PROC; import static org.apache.ignite.internal.IgniteComponentType.JTA; +import static org.apache.ignite.internal.IgniteFeatures.LRT_SYSTEM_USER_TIME_DUMP_SETTINGS; import static org.apache.ignite.internal.IgniteFeatures.TRANSACTION_OWNER_THREAD_DUMP_PROVIDING; import static org.apache.ignite.internal.processors.cache.GridCacheUtils.isNearEnabled; import static org.apache.ignite.internal.processors.cache.GridCacheUtils.isPersistentCache; @@ -5183,6 +5185,71 @@ public class GridCacheProcessor extends GridProcessorAdapter { } /** + * Sets threshold timeout in milliseconds for long transactions, if transaction exceeds it, + * it will be dumped in log with information about how much time did + * it spent in system time (time while aquiring locks, preparing, commiting, etc.) + * and user time (time when client node runs some code while holding transaction). + * Can be set to 0 - no transactions will be dumped in log in this case. + * + * @param threshold Threshold timeout in milliseconds. + */ + public void longTransactionTimeDumpThreshold(long threshold) { + assert threshold >= 0 : "Threshold timeout must be greater than or equal to 0."; + + broadcastToNodesSupportingFeature( + new LongRunningTxTimeDumpSettingsClosure(threshold, null, null), + LRT_SYSTEM_USER_TIME_DUMP_SETTINGS + ); + } + + /** + * Sets the coefficient for samples of long running transactions that will be dumped in log, if + * {@link #longTransactionTimeDumpThreshold} is set to non-zero value." + * + * @param coefficient Coefficient, must be value between 0.0 and 1.0 inclusively. + */ + public void transactionTimeDumpSamplesCoefficient(double coefficient) { + assert coefficient >= 0.0 && coefficient <= 1.0 : "Percentage value must be between 0.0 and 1.0 inclusively."; + + broadcastToNodesSupportingFeature( + new LongRunningTxTimeDumpSettingsClosure(null, coefficient, null), + LRT_SYSTEM_USER_TIME_DUMP_SETTINGS + ); + } + + /** + * Sets the limit of samples of completed transactions that will be dumped in log per second, + * if {@link #transactionTimeDumpSamplesCoefficient} is above <code>0.0</code>. + * Must be integer value greater than <code>0</code>. + * + * @param limit Limit value. + */ + public void longTransactionTimeDumpSamplesPerSecondLimit(int limit) { + assert limit > 0 : "Limit value must be greater than 0."; + + broadcastToNodesSupportingFeature( + new LongRunningTxTimeDumpSettingsClosure(null, null, limit), + LRT_SYSTEM_USER_TIME_DUMP_SETTINGS + ); + } + + /** + * Broadcasts given job to nodes that support ignite feature. + * + * @param job Ignite job. + * @param feature Ignite feature. + */ + private void broadcastToNodesSupportingFeature(IgniteRunnable job, IgniteFeatures feature) { + ClusterGroup grp = ctx.grid() + .cluster() + .forPredicate(node -> IgniteFeatures.nodeSupports(node, feature)); + + IgniteCompute compute = ctx.grid().compute(grp); + + compute.broadcast(job); + } + + /** * @param oldFormat Old format. */ private CacheConfigurationSplitter splitter(boolean oldFormat) { diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/LongRunningTxTimeDumpSettingsClosure.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/LongRunningTxTimeDumpSettingsClosure.java new file mode 100644 index 0000000..95c6ea0 --- /dev/null +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/LongRunningTxTimeDumpSettingsClosure.java @@ -0,0 +1,74 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.cache; + +import org.apache.ignite.Ignite; +import org.apache.ignite.internal.IgniteEx; +import org.apache.ignite.internal.processors.cache.transactions.IgniteTxManager; +import org.apache.ignite.lang.IgniteRunnable; +import org.apache.ignite.resources.IgniteInstanceResource; + +/** + * Closure that is sent on all server nodes in order to change configuration parameters + * of dumping long running transactions' system and user time values. + */ +public class LongRunningTxTimeDumpSettingsClosure implements IgniteRunnable { + /** */ + private static final long serialVersionUID = 0L; + + /** */ + private final Long timeoutThreshold; + + /** */ + private final Double samplesCoefficient; + + /** */ + private final Integer samplesPerSecondLimit; + + /** + * Auto-inject Ignite instance + */ + @IgniteInstanceResource + private Ignite ignite; + + /** */ + public LongRunningTxTimeDumpSettingsClosure( + Long timeoutThreshold, + Double samplesCoefficient, + Integer samplesPerSecondLimit + ) { + this.timeoutThreshold = timeoutThreshold; + this.samplesCoefficient = samplesCoefficient; + this.samplesPerSecondLimit = samplesPerSecondLimit; + } + + /** {@inheritDoc} */ + @Override + public void run() { + IgniteTxManager tm = ((IgniteEx) ignite).context().cache().context().tm(); + + if (timeoutThreshold != null) + tm.longTransactionTimeDumpThreshold(timeoutThreshold); + + if (samplesCoefficient != null) + tm.transactionTimeDumpSamplesCoefficient(samplesCoefficient); + + if (samplesPerSecondLimit != null) + tm.transactionTimeDumpSamplesPerSecondLimit(samplesPerSecondLimit); + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java index 858682c..65ef567 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java @@ -18,9 +18,11 @@ package org.apache.ignite.internal.processors.cache.distributed.near; import java.io.Externalizable; +import java.text.SimpleDateFormat; import java.util.ArrayList; import java.util.Collection; import java.util.Collections; +import java.util.Date; import java.util.HashMap; import java.util.HashSet; import java.util.Iterator; @@ -28,6 +30,8 @@ import java.util.LinkedHashMap; import java.util.Map; import java.util.Set; import java.util.UUID; +import java.util.concurrent.ThreadLocalRandom; +import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReferenceFieldUpdater; import javax.cache.Cache; import javax.cache.CacheException; @@ -74,6 +78,7 @@ import org.apache.ignite.internal.processors.cache.mvcc.MvccUtils; import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx; import org.apache.ignite.internal.processors.cache.transactions.IgniteTxEntry; import org.apache.ignite.internal.processors.cache.transactions.IgniteTxKey; +import org.apache.ignite.internal.processors.cache.transactions.IgniteTxManager; import org.apache.ignite.internal.processors.cache.transactions.TransactionProxy; import org.apache.ignite.internal.processors.cache.transactions.TransactionProxyImpl; import org.apache.ignite.internal.processors.cache.transactions.TransactionProxyRollbackOnlyImpl; @@ -85,6 +90,7 @@ import org.apache.ignite.internal.transactions.IgniteTxOptimisticCheckedExceptio import org.apache.ignite.internal.transactions.IgniteTxRollbackCheckedException; import org.apache.ignite.internal.transactions.IgniteTxTimeoutCheckedException; import org.apache.ignite.internal.util.GridLeanMap; +import org.apache.ignite.internal.util.GridStringBuilder; import org.apache.ignite.internal.util.IgniteUtils; import org.apache.ignite.internal.util.future.GridEmbeddedFuture; import org.apache.ignite.internal.util.future.GridFinishedFuture; @@ -139,6 +145,10 @@ public class GridNearTxLocal extends GridDhtTxLocalAdapter implements GridTimeou /** */ private static final long serialVersionUID = 0L; + /** */ + private static final ThreadLocal<SimpleDateFormat> TIME_FORMAT = + ThreadLocal.withInitial(() -> new SimpleDateFormat("HH:mm:ss.SSS")); + /** Prepare future updater. */ private static final AtomicReferenceFieldUpdater<GridNearTxLocal, IgniteInternalFuture> PREP_FUT_UPD = AtomicReferenceFieldUpdater.newUpdater(GridNearTxLocal.class, IgniteInternalFuture.class, "prepFut"); @@ -183,6 +193,40 @@ public class GridNearTxLocal extends GridDhtTxLocalAdapter implements GridTimeou /** */ private boolean trackTimeout; + /** + * Counts how much time this transaction has spent on system calls, in nanoseconds. + */ + private final AtomicLong systemTime = new AtomicLong(0); + + /** + * Stores the nano time value when current system time has started, or <code>0</code> if no system section + * is running currently. + */ + private final AtomicLong systemStartTime = new AtomicLong(0); + + /** + * Stores the nano time value when prepare step has started, or <code>0</code> if no prepare step + * has started yet. + */ + private final AtomicLong prepareStartTime = new AtomicLong(0); + + /** + * Stores prepare step duration, or <code>0</code> if it has not finished yet. + */ + private final AtomicLong prepareTime = new AtomicLong(0); + + /** + * Stores the nano time value when commit or rollback step has started, or <code>0</code> if it + * has not started yet. + */ + private final AtomicLong commitOrRollbackStartTime = new AtomicLong(0); + + /** Stores commit or rollback step duration, or <code>0</code> if it has not finished yet. */ + private final AtomicLong commitOrRollbackTime = new AtomicLong(0); + + /** */ + private IgniteTxManager.TxDumpsThrottling txDumpsThrottling; + /** */ @GridToStringExclude private TransactionProxyImpl proxy; @@ -228,6 +272,7 @@ public class GridNearTxLocal extends GridDhtTxLocalAdapter implements GridTimeou * @param subjId Subject ID. * @param taskNameHash Task name hash code. * @param lb Label. + * @param txDumpsThrottling Log throttling information. */ public GridNearTxLocal( GridCacheSharedContext ctx, @@ -243,7 +288,8 @@ public class GridNearTxLocal extends GridDhtTxLocalAdapter implements GridTimeou int txSize, @Nullable UUID subjId, int taskNameHash, - @Nullable String lb + @Nullable String lb, + IgniteTxManager.TxDumpsThrottling txDumpsThrottling ) { super( ctx, @@ -268,6 +314,8 @@ public class GridNearTxLocal extends GridDhtTxLocalAdapter implements GridTimeou this.mvccOp = mvccOp; + this.txDumpsThrottling = txDumpsThrottling; + initResult(); trackTimeout = timeout() > 0 && !implicit() && cctx.time().addTimeoutObject(this); @@ -3847,9 +3895,130 @@ public class GridNearTxLocal extends GridDhtTxLocalAdapter implements GridTimeou } /** + * Returns current amount of time that transaction has spent on system activities (acquiring locks, commiting, + * rolling back, etc.) + * + * @return Amount of time in milliseconds. + */ + public long systemTimeCurrent() { + long systemTime0 = systemTime.get(); + + long systemStartTime0 = systemStartTime.get(); + + long t = systemStartTime0 == 0 ? 0 : (System.nanoTime() - systemStartTime0); + + return U.nanosToMillis(systemTime0 + t); + } + + /** {@inheritDoc} */ + @Override public boolean state(TransactionState state) { + boolean res = super.state(state); + + if (state == COMMITTED || state == ROLLED_BACK) { + leaveSystemSection(); + + //if commitOrRollbackTime != 0 it means that we already have written metrics and dumped it in log at least once + if (!commitOrRollbackTime.compareAndSet(0, System.nanoTime() - commitOrRollbackStartTime.get())) + return res; + + long systemTimeMillis = U.nanosToMillis(this.systemTime.get()); + long totalTimeMillis = System.currentTimeMillis() - startTime(); + + //in some cases totalTimeMillis can be less than systemTimeMillis, as they are calculated with different precision + long userTimeMillis = Math.max(totalTimeMillis - systemTimeMillis, 0); + + cctx.txMetrics().onNearTxComplete(systemTimeMillis, userTimeMillis); + + boolean willBeSkipped = txDumpsThrottling == null || txDumpsThrottling.skipCurrent(); + + if (!willBeSkipped) { + long transactionTimeDumpThreshold = cctx.tm().longTransactionTimeDumpThreshold(); + + double transactionTimeDumpSamplesCoefficient = cctx.tm().transactionTimeDumpSamplesCoefficient(); + + boolean isLong = transactionTimeDumpThreshold > 0 && totalTimeMillis > transactionTimeDumpThreshold; + + boolean randomlyChosen = transactionTimeDumpSamplesCoefficient > 0.0 + && ThreadLocalRandom.current().nextDouble() <= transactionTimeDumpSamplesCoefficient; + + if (randomlyChosen || isLong) { + String txDump = completedTransactionDump(state, systemTimeMillis, userTimeMillis, isLong); + + if (isLong) + log.warning(txDump); + else + log.info(txDump); + + txDumpsThrottling.dump(); + } + } + else if (txDumpsThrottling != null) + txDumpsThrottling.skip(); + } + + return res; + } + + /** + * Builds dump string for completed transaction. + * + * @param state Transaction state. + * @param systemTimeMillis System time in milliseconds. + * @param userTimeMillis User time in milliseconds. + * @param isLong Whether the dumped transaction is long running or not. + * @return Dump string. + */ + private String completedTransactionDump( + TransactionState state, + long systemTimeMillis, + long userTimeMillis, + boolean isLong + ) { + long cacheOperationsTimeMillis = + U.nanosToMillis(systemTime.get() - prepareTime.get() - commitOrRollbackTime.get()); + + GridStringBuilder warning = new GridStringBuilder(isLong ? "Long transaction time dump " : "Transaction time dump ") + .a("[startTime=") + .a(TIME_FORMAT.get().format(new Date(startTime))) + .a(", totalTime=") + .a(systemTimeMillis + userTimeMillis) + .a(", systemTime=") + .a(systemTimeMillis) + .a(", userTime=") + .a(userTimeMillis) + .a(", cacheOperationsTime=") + .a(cacheOperationsTimeMillis); + + if (state == COMMITTED) { + warning + .a(", prepareTime=") + .a(timeMillis(prepareTime)) + .a(", commitTime=") + .a(timeMillis(commitOrRollbackTime)); + } + else { + warning + .a(", rollbackTime=") + .a(timeMillis(commitOrRollbackTime)); + } + + warning + .a(", tx=") + .a(this) + .a("]"); + + return warning.toString(); + } + + /** * @return Tx prepare future. */ public IgniteInternalFuture<?> prepareNearTxLocal() { + enterSystemSection(); + + //we assume that prepare start time should be set only once for the transaction + prepareStartTime.compareAndSet(0, System.nanoTime()); + GridNearTxPrepareFutureAdapter fut = (GridNearTxPrepareFutureAdapter)prepFut; if (fut == null) { @@ -3948,6 +4117,11 @@ public class GridNearTxLocal extends GridDhtTxLocalAdapter implements GridTimeou prepareFut.listen(new CI1<IgniteInternalFuture<?>>() { @Override public void apply(IgniteInternalFuture<?> f) { + //these values should not be changed after set once + prepareTime.compareAndSet(0, System.nanoTime() - prepareStartTime.get()); + + commitOrRollbackStartTime.compareAndSet(0, System.nanoTime()); + try { // Make sure that here are no exceptions. f.get(); @@ -4010,6 +4184,8 @@ public class GridNearTxLocal extends GridDhtTxLocalAdapter implements GridTimeou if (log.isDebugEnabled()) log.debug("Rolling back near tx: " + this); + enterSystemSection(); + if (!onTimeout && trackTimeout) removeTimeoutHandler(); @@ -4961,6 +5137,30 @@ public class GridNearTxLocal extends GridDhtTxLocalAdapter implements GridTimeou } } + /** */ + private long timeMillis(AtomicLong atomicNanoTime) { + return U.nanosToMillis(atomicNanoTime.get()); + } + + /** + * Enters the section when system time for this transaction is counted. + */ + public void enterSystemSection() { + //setting systemStartTime only if it equals 0, otherwise it means that we are already in system section + //and sould do nothing. + systemStartTime.compareAndSet(0, System.nanoTime()); + } + + /** + * Leaves the section when system time for this transaction is counted. + */ + public void leaveSystemSection() { + long systemStartTime0 = systemStartTime.getAndSet(0); + + if (systemStartTime0 > 0) + systemTime.addAndGet(System.nanoTime() - systemStartTime0); + } + /** * Post-lock closure. * diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxManager.java index d73dc39..7ba442a 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxManager.java @@ -30,6 +30,7 @@ import java.util.Set; import java.util.UUID; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.atomic.AtomicInteger; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteClientDisconnectedException; import org.apache.ignite.IgniteException; @@ -77,6 +78,7 @@ import org.apache.ignite.internal.processors.cache.mvcc.msg.MvccRecoveryFinished import org.apache.ignite.internal.processors.cache.transactions.TxDeadlockDetection.TxDeadlockFuture; import org.apache.ignite.internal.processors.cache.version.GridCacheVersion; import org.apache.ignite.internal.processors.cluster.BaselineTopology; +import org.apache.ignite.internal.processors.metric.impl.HitRateMetric; import org.apache.ignite.internal.processors.timeout.GridTimeoutObjectAdapter; import org.apache.ignite.internal.transactions.IgniteTxOptimisticCheckedException; import org.apache.ignite.internal.transactions.IgniteTxRollbackCheckedException; @@ -101,8 +103,11 @@ import org.jsr166.ConcurrentLinkedHashMap; import static org.apache.ignite.IgniteSystemProperties.IGNITE_DEFERRED_ONE_PHASE_COMMIT_ACK_REQUEST_BUFFER_SIZE; import static org.apache.ignite.IgniteSystemProperties.IGNITE_DEFERRED_ONE_PHASE_COMMIT_ACK_REQUEST_TIMEOUT; +import static org.apache.ignite.IgniteSystemProperties.IGNITE_LONG_TRANSACTION_TIME_DUMP_THRESHOLD; import static org.apache.ignite.IgniteSystemProperties.IGNITE_MAX_COMPLETED_TX_COUNT; import static org.apache.ignite.IgniteSystemProperties.IGNITE_SLOW_TX_WARN_TIMEOUT; +import static org.apache.ignite.IgniteSystemProperties.IGNITE_TRANSACTION_TIME_DUMP_SAMPLES_COEFFICIENT; +import static org.apache.ignite.IgniteSystemProperties.IGNITE_TRANSACTION_TIME_DUMP_SAMPLES_PER_SECOND_LIMIT; import static org.apache.ignite.IgniteSystemProperties.IGNITE_TX_DEADLOCK_DETECTION_MAX_ITERS; import static org.apache.ignite.IgniteSystemProperties.IGNITE_TX_OWNER_DUMP_REQUESTS_ALLOWED; import static org.apache.ignite.IgniteSystemProperties.IGNITE_TX_SALVAGE_TIMEOUT; @@ -196,6 +201,29 @@ public class IgniteTxManager extends GridCacheSharedManagerAdapter { private boolean txOwnerDumpRequestsAllowed = IgniteSystemProperties.getBoolean(IGNITE_TX_OWNER_DUMP_REQUESTS_ALLOWED, true); + /** + * Threshold timeout for long transactions, if transaction exceeds it, it will be dumped in log with + * information about how much time did it spent in system time (time while aquiring locks, preparing, + * commiting, etc) and user time (time when client node runs some code while holding transaction and not + * waiting it). Equals 0 if not set. No transactions are dumped in log if this parameter is not set. + */ + private volatile long longTransactionTimeDumpThreshold = + IgniteSystemProperties.getLong(IGNITE_LONG_TRANSACTION_TIME_DUMP_THRESHOLD, 0); + + /** + * The coefficient for samples of completed transactions that will be dumped in log. + */ + private volatile double transactionTimeDumpSamplesCoefficient = + IgniteSystemProperties.getFloat(IGNITE_TRANSACTION_TIME_DUMP_SAMPLES_COEFFICIENT, 0.0f); + + /** + * The limit of samples of completed transactions that will be dumped in log per second, if + * {@link #transactionTimeDumpSamplesCoefficient} is above <code>0.0</code>. Must be integer value + * greater than <code>0</code>. + */ + private volatile int longTransactionTimeDumpSamplesPerSecondLimit = + IgniteSystemProperties.getInteger(IGNITE_TRANSACTION_TIME_DUMP_SAMPLES_PER_SECOND_LIMIT, 5); + /** Committed local transactions. */ private final GridBoundedConcurrentOrderedMap<GridCacheVersion, Boolean> completedVersSorted = new GridBoundedConcurrentOrderedMap<>( @@ -219,6 +247,9 @@ public class IgniteTxManager extends GridCacheSharedManagerAdapter { /** Long operations dump timeout. */ private long longOpsDumpTimeout = LONG_OPERATIONS_DUMP_TIMEOUT; + /** */ + private TxDumpsThrottling txDumpsThrottling = new TxDumpsThrottling(); + /** * Near version to DHT version map. Note that we initialize to 5K size from get go, * to avoid future map resizings. @@ -478,6 +509,71 @@ public class IgniteTxManager extends GridCacheSharedManagerAdapter { } /** + * Threshold timeout for long transactions, if transaction exceeds it, it will be dumped in log with + * information about how much time did it spent in system time (time while aquiring locks, preparing, + * commiting, etc) and user time (time when client node runs some code while holding transaction and not + * waiting it). Equals 0 if not set. No transactions are dumped in log if this parameter is not set. + * + * @return Threshold timeout in milliseconds. + */ + public long longTransactionTimeDumpThreshold() { + return longTransactionTimeDumpThreshold; + } + + /** + * Sets threshold timeout for long transactions, if transaction exceeds it, it will be dumped in log with + * information about how much time did it spent in system time (time while aquiring locks, preparing, + * commiting, etc) and user time (time when client node runs some code while holding transaction and not + * waiting it). Can be set to 0 - no transactions will be dumped in log in this case. + * + * @param longTransactionTimeDumpThreshold Value of threshold timeout in milliseconds. + */ + public void longTransactionTimeDumpThreshold(long longTransactionTimeDumpThreshold) { + assert longTransactionTimeDumpThreshold >= 0 + : "longTransactionTimeDumpThreshold must be greater than or equal to 0."; + + this.longTransactionTimeDumpThreshold = longTransactionTimeDumpThreshold; + } + + /** + * The coefficient for samples of completed transactions that will be dumped in log. + */ + public double transactionTimeDumpSamplesCoefficient() { + return transactionTimeDumpSamplesCoefficient; + } + + /** + * Sets the coefficient for samples of completed transactions that will be dumped in log. + */ + public void transactionTimeDumpSamplesCoefficient(double transactionTimeDumpSamplesCoefficient) { + assert transactionTimeDumpSamplesCoefficient >= 0.0 && transactionTimeDumpSamplesCoefficient <= 1.0 + : "transactionTimeDumpSamplesCoefficient value must be between 0.0 and 1.0 inclusively."; + + this.transactionTimeDumpSamplesCoefficient = transactionTimeDumpSamplesCoefficient; + } + + /** + * The limit of samples of completed transactions that will be dumped in log per second, if + * {@link #transactionTimeDumpSamplesCoefficient} is above <code>0.0</code>. Must be integer value + * greater than <code>0</code>. + */ + public int transactionTimeDumpSamplesPerSecondLimit() { + return longTransactionTimeDumpSamplesPerSecondLimit; + } + + /** + * Sets the limit of samples of completed transactions that will be dumped in log per second, if + * {@link #transactionTimeDumpSamplesCoefficient} is above <code>0.0</code>. Must be integer value + * greater than <code>0</code>. + */ + public void transactionTimeDumpSamplesPerSecondLimit(int transactionTimeDumpSamplesPerSecondLimit) { + assert transactionTimeDumpSamplesPerSecondLimit > 0 + : "transactionTimeDumpSamplesPerSecondLimit must be integer value greater than 0."; + + this.longTransactionTimeDumpSamplesPerSecondLimit = transactionTimeDumpSamplesPerSecondLimit; + } + + /** * Invalidates transaction. * * @param tx Transaction. @@ -608,7 +704,9 @@ public class IgniteTxManager extends GridCacheSharedManagerAdapter { txSize, subjId, taskNameHash, - lb); + lb, + txDumpsThrottling + ); if (tx.system()) { AffinityTopologyVersion topVer = cctx.tm().lockedTopologyVersion(Thread.currentThread().getId(), tx); @@ -1671,6 +1769,27 @@ public class IgniteTxManager extends GridCacheSharedManagerAdapter { } /** + * Enters system section for thread local near tx, if it is present. + * In this section system time for this transaction is counted. + */ + public void enterNearTxSystemSection() { + GridNearTxLocal tx = threadLocalTx(null); + + if (tx != null) + tx.enterSystemSection(); + } + + /** + * Leaves system section for thread local near tx, if it is present. + */ + public void leaveNearTxSystemSection() { + GridNearTxLocal tx = threadLocalTx(null); + + if (tx != null) + tx.leaveSystemSection(); + } + + /** * Gets transaction ID map depending on transaction type. * * @param tx Transaction. @@ -2985,4 +3104,46 @@ public class IgniteTxManager extends GridCacheSharedManagerAdapter { } } } + + /** + * This class is used to store information about transaction time dump throttling. + */ + public class TxDumpsThrottling { + /** */ + private AtomicInteger skippedTxCntr = new AtomicInteger(); + + /** */ + private HitRateMetric transactionHitRateCntr = new HitRateMetric("transactionHitRateCounter", null, 1000, 2); + + /** + * Returns should we skip dumping the transaction in current moment. + */ + public boolean skipCurrent() { + boolean res = transactionHitRateCntr.value() >= transactionTimeDumpSamplesPerSecondLimit(); + + if (!res) { + int skipped = skippedTxCntr.getAndSet(0); + + //we should not log info about skipped dumps if skippedTxCounter was reset concurrently + if (skipped > 0) + log.info("Transaction time dumps skipped because of log throttling: " + skipped); + } + + return res; + } + + /** + * Should be called when we dump transaction to log. + */ + public void dump() { + transactionHitRateCntr.increment(); + } + + /** + * Should be called when we skip transaction which we could dump to log because of throttling. + */ + public void skip() { + skippedTxCntr.incrementAndGet(); + } + } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/TransactionMetricsAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/TransactionMetricsAdapter.java index ce85a85..6193065 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/TransactionMetricsAdapter.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/TransactionMetricsAdapter.java @@ -32,7 +32,9 @@ import org.apache.ignite.internal.processors.cache.GridCacheMvccManager; import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxLocal; import org.apache.ignite.internal.processors.metric.MetricRegistry; import org.apache.ignite.internal.processors.metric.impl.AtomicLongMetric; +import org.apache.ignite.internal.processors.metric.impl.HistogramMetric; import org.apache.ignite.internal.processors.metric.impl.IntMetricImpl; +import org.apache.ignite.internal.processors.metric.impl.LongAdderMetric; import org.apache.ignite.internal.util.GridStringBuilder; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.internal.S; @@ -48,6 +50,22 @@ import static org.apache.ignite.internal.processors.metric.GridMetricManager.TX_ * Tx metrics adapter. */ public class TransactionMetricsAdapter implements TransactionMetrics { + /** Metric name for total system time on node. */ + public static final String METRIC_TOTAL_SYSTEM_TIME = "totalNodeSystemTime"; + + /** Metric name for system time histogram on node. */ + public static final String METRIC_SYSTEM_TIME_HISTOGRAM = "nodeSystemTimeHistogram"; + + /** Metric name for total user time on node. */ + public static final String METRIC_TOTAL_USER_TIME = "totalNodeUserTime"; + + /** Metric name for user time histogram on node. */ + public static final String METRIC_USER_TIME_HISTOGRAM = "nodeUserTimeHistogram"; + + /** Histogram buckets for metrics of system and user time. */ + public static final long[] METRIC_TIME_BUCKETS = + new long[] { 1, 2, 4, 8, 16, 25, 50, 75, 100, 250, 500, 750, 1000, 3000, 5000, 10000, 25000, 60000}; + /** Grid kernal context. */ private final GridKernalContext gridKernalCtx; @@ -63,6 +81,18 @@ public class TransactionMetricsAdapter implements TransactionMetrics { /** Last rollback time. */ private final AtomicLongMetric rollbackTime; + /** Holds the reference to metric for total system time on node.*/ + private LongAdderMetric totalTxSystemTime; + + /** Holds the reference to metric for total user time on node. */ + private LongAdderMetric totalTxUserTime; + + /** Holds the reference to metric for system time histogram on node. */ + private HistogramMetric txSystemTimeHistogram; + + /** Holds the reference to metric for user time histogram on node. */ + private HistogramMetric txUserTimeHistogram; + /** * @param ctx Kernal context. */ @@ -75,6 +105,20 @@ public class TransactionMetricsAdapter implements TransactionMetrics { txRollbacks = mreg.intMetric("txRollbacks", "Number of transaction rollbacks."); commitTime = mreg.longMetric("commitTime", "Last commit time."); rollbackTime = mreg.longMetric("rollbackTime", "Last rollback time."); + totalTxSystemTime = mreg.longAdderMetric(METRIC_TOTAL_SYSTEM_TIME, "Total transactions system time on node."); + totalTxUserTime = mreg.longAdderMetric(METRIC_TOTAL_USER_TIME, "Total transactions user time on node."); + + txSystemTimeHistogram = mreg.histogram( + METRIC_SYSTEM_TIME_HISTOGRAM, + METRIC_TIME_BUCKETS, + "Transactions system times on node represented as histogram." + ); + + txUserTimeHistogram = mreg.histogram( + METRIC_USER_TIME_HISTOGRAM, + METRIC_TIME_BUCKETS, + "Transactions user times on node represented as histogram." + ); mreg.register("AllOwnerTransactions", this::getAllOwnerTransactions, @@ -168,6 +212,26 @@ public class TransactionMetricsAdapter implements TransactionMetrics { } /** + * Callback for completion of near transaction. Writes metrics of single near transaction. + * + * @param systemTime Transaction system time. + * @param userTime Transaction user time. + */ + public void onNearTxComplete(long systemTime, long userTime) { + if (systemTime >= 0) { + totalTxSystemTime.add(systemTime); + + txSystemTimeHistogram.value(systemTime); + } + + if (userTime >= 0) { + totalTxUserTime.add(userTime); + + txUserTimeHistogram.value(userTime); + } + } + + /** * Reset. */ public void reset() { diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/TransactionProxyImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/TransactionProxyImpl.java index 110f34d..486d9ba 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/TransactionProxyImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/TransactionProxyImpl.java @@ -116,6 +116,8 @@ public class TransactionProxyImpl<K, V> implements TransactionProxy, Externaliza if (cctx.deploymentEnabled()) cctx.deploy().onEnter(); + tx.enterSystemSection(); + try { cctx.kernalContext().gateway().readLock(); } @@ -135,6 +137,8 @@ public class TransactionProxyImpl<K, V> implements TransactionProxy, Externaliza private void leave() { try { CU.unwindEvicts(cctx); + + tx.leaveSystemSection(); } finally { cctx.kernalContext().gateway().readUnlock(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/metric/impl/HistogramMetric.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/metric/impl/HistogramMetric.java index 2d0304d..173122a 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/metric/impl/HistogramMetric.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/metric/impl/HistogramMetric.java @@ -91,6 +91,17 @@ public class HistogramMetric extends AbstractMetric implements ObjectMetric<long return res; } + /** + * Returns bounds of this histogram. + */ + public long[] bounds() { + long[] res = new long[holder.bounds.length]; + + System.arraycopy(holder.bounds, 0, res, 0, res.length); + + return res; + } + /** {@inheritDoc} */ @Override public Class<long[]> type() { return long[].class; diff --git a/modules/core/src/main/java/org/apache/ignite/mxbean/TransactionsMXBean.java b/modules/core/src/main/java/org/apache/ignite/mxbean/TransactionsMXBean.java index eef7931..4efef27 100644 --- a/modules/core/src/main/java/org/apache/ignite/mxbean/TransactionsMXBean.java +++ b/modules/core/src/main/java/org/apache/ignite/mxbean/TransactionsMXBean.java @@ -128,4 +128,94 @@ public interface TransactionsMXBean { "whether to allow" ) public void setTxOwnerDumpRequestsAllowed(boolean allowed); + + /** + * Returns threshold timeout in milliseconds for long transactions, if transaction exceeds it, + * it will be dumped in log with information about how much time did + * it spent in system time (time while aquiring locks, preparing, commiting, etc.) + * and user time (time when client node runs some code while holding transaction). + * Returns 0 if not set. No transactions are dumped in log if this parameter is not set. + * + * @return Threshold. + */ + @MXBeanDescription( + "Returns threshold timeout in milliseconds for long transactions, if transaction exceeds it, " + + "it will be dumped in log with information about how much time did " + + "it spent in system time (time while aquiring locks, preparing, commiting, etc.)" + + "and user time (time when client node runs some code while holding transaction). " + + "Returns 0 if not set. No transactions are dumped in log if this parameter is not set." + ) + public long getLongTransactionTimeDumpThreshold(); + + /** + * Sets threshold timeout in milliseconds for long transactions, if transaction exceeds it, + * it will be dumped in log with information about how much time did + * it spent in system time (time while aquiring locks, preparing, commiting, etc.) + * and user time (time when client node runs some code while holding transaction). + * Can be set to 0 - no transactions will be dumped in log in this case. + * + * @param threshold Threshold. + */ + @MXBeanDescription( + "Sets threshold timeout in milliseconds for long transactions, if transaction exceeds it, " + + "it will be dumped in log with information about how much time did " + + "it spent in system time (time while aquiring locks, preparing, commiting, etc.) " + + "and user time (time when client node runs some code while holding transaction). " + + "Can be set to 0 - no transactions will be dumped in log in this case." + ) + @MXBeanParametersNames("threshold") + @MXBeanParametersDescriptions("threshold timeout") + public void setLongTransactionTimeDumpThreshold(long threshold); + + /** + * Returns the coefficient for samples of completed transactions that will be dumped in log. + * + * @return Coefficient current value. + */ + @MXBeanDescription( + "Returns the coefficient for samples of completed transactions that will be dumped in log." + ) + public double getTransactionTimeDumpSamplesCoefficient(); + + /** + * Sets the coefficient for samples of completed transactions that will be dumped in log. + * + * @param coefficient Coefficient. + */ + @MXBeanDescription( + "Sets the coefficient for samples of completed transactions that will be dumped in log." + ) + @MXBeanParametersNames("coefficient") + @MXBeanParametersDescriptions("Samples coefficient.") + public void setTransactionTimeDumpSamplesCoefficient(double coefficient); + + /** + * Returns the limit of samples of completed transactions that will be dumped in log per second, + * if {@link #getTransactionTimeDumpSamplesCoefficient} is above <code>0.0</code>. + * Must be integer value greater than <code>0</code>. + * + * @return Limit value. + */ + @MXBeanDescription( + "Returns the limit of samples of completed transactions that will be dumped in log per second, " + + "if {@link #getTransactionTimeDumpSamplesCoefficient} is above <code>0.0</code>. " + + "Must be integer value greater than <code>0</code>." + ) + public int getTransactionTimeDumpSamplesPerSecondLimit(); + + /** + * Sets the limit of samples of completed transactions that will be dumped in log per second, + * if {@link #getTransactionTimeDumpSamplesCoefficient} is above <code>0.0</code>. + * Must be integer value greater than <code>0</code>. + * + * @param limit Limit value. + */ + @MXBeanDescription( + "Sets the limit of samples of completed transactions that will be dumped in log per second, " + + "if {@link #getTransactionTimeDumpSamplesCoefficient} is above <code>0.0</code>. " + + "Must be integer value greater than <code>0</code>." + ) + @MXBeanParametersNames("limit") + @MXBeanParametersDescriptions("Samples per second limit.") + public void setTransactionTimeDumpSamplesPerSecondLimit(int limit); } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/metric/JmxMetricExporterSpiTest.java b/modules/core/src/test/java/org/apache/ignite/internal/metric/JmxMetricExporterSpiTest.java index dae1ba3..8c50a54 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/metric/JmxMetricExporterSpiTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/metric/JmxMetricExporterSpiTest.java @@ -17,22 +17,16 @@ package org.apache.ignite.internal.metric; -import java.lang.management.ManagementFactory; import java.util.Optional; import java.util.Set; import javax.management.DynamicMBean; import javax.management.MBeanAttributeInfo; import javax.management.MBeanFeatureInfo; -import javax.management.MBeanServer; -import javax.management.MBeanServerInvocationHandler; -import javax.management.MalformedObjectNameException; -import javax.management.ObjectName; import org.apache.ignite.IgniteException; import org.apache.ignite.configuration.DataRegionConfiguration; import org.apache.ignite.configuration.DataStorageConfiguration; import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.internal.IgniteEx; -import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.spi.metric.jmx.JmxMetricExporterSpi; import org.apache.ignite.testframework.GridTestUtils.RunnableX; import org.junit.Test; @@ -87,7 +81,7 @@ public class JmxMetricExporterSpiTest extends AbstractExporterSpiTest { /** */ @Test public void testSysJmxMetrics() throws Exception { - DynamicMBean sysMBean = metricSet(null, SYS_METRICS); + DynamicMBean sysMBean = metricSet(ignite.name(), null, SYS_METRICS); Set<String> res = stream(sysMBean.getMBeanInfo().getAttributes()) .map(MBeanFeatureInfo::getName) @@ -118,7 +112,7 @@ public class JmxMetricExporterSpiTest extends AbstractExporterSpiTest { /** */ @Test public void testDataRegionJmxMetrics() throws Exception { - DynamicMBean dataRegionMBean = metricSet("io", "dataregion.default"); + DynamicMBean dataRegionMBean = metricSet(ignite.name(), "io", "dataregion.default"); Set<String> res = stream(dataRegionMBean.getMBeanInfo().getAttributes()) .map(MBeanFeatureInfo::getName) @@ -137,29 +131,17 @@ public class JmxMetricExporterSpiTest extends AbstractExporterSpiTest { assertThrowsWithCause(new RunnableX() { @Override public void runx() throws Exception { - metricSet("filtered", "metric"); + metricSet(ignite.name(), "filtered", "metric"); } }, IgniteException.class); - DynamicMBean bean1 = metricSet("other", "prefix"); + DynamicMBean bean1 = metricSet(ignite.name(), "other", "prefix"); assertEquals(42L, bean1.getAttribute("test")); assertEquals(43L, bean1.getAttribute("test2")); - DynamicMBean bean2 = metricSet("other", "prefix2"); + DynamicMBean bean2 = metricSet(ignite.name(), "other", "prefix2"); assertEquals(44L, bean2.getAttribute("test3")); } - - /** */ - public DynamicMBean metricSet(String grp, String name) throws MalformedObjectNameException { - ObjectName mbeanName = U.makeMBeanName(ignite.name(), grp, name); - - MBeanServer mbeanSrv = ManagementFactory.getPlatformMBeanServer(); - - if (!mbeanSrv.isRegistered(mbeanName)) - throw new IgniteException("MBean not registered."); - - return MBeanServerInvocationHandler.newProxyInstance(mbeanSrv, mbeanName, DynamicMBean.class, false); - } } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridTransactionsSystemUserTimeMetricsTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridTransactionsSystemUserTimeMetricsTest.java new file mode 100644 index 0000000..8e6d038 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridTransactionsSystemUserTimeMetricsTest.java @@ -0,0 +1,327 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.ignite.internal.processors.cache; + +import java.util.concurrent.Callable; +import java.util.concurrent.atomic.AtomicInteger; +import javax.management.DynamicMBean; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.IgniteException; +import org.apache.ignite.IgniteLogger; +import org.apache.ignite.cluster.ClusterNode; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.internal.TransactionsMXBeanImpl; +import org.apache.ignite.internal.managers.communication.GridIoMessage; +import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxPrepareRequest; +import org.apache.ignite.internal.processors.cache.transactions.IgniteTxAdapter; +import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.lang.IgniteInClosure; +import org.apache.ignite.mxbean.TransactionsMXBean; +import org.apache.ignite.plugin.extensions.communication.Message; +import org.apache.ignite.spi.IgniteSpiException; +import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi; +import org.apache.ignite.spi.metric.jmx.JmxMetricExporterSpi; +import org.apache.ignite.testframework.GridTestUtils; +import org.apache.ignite.testframework.ListeningTestLogger; +import org.apache.ignite.testframework.LogListener; +import org.apache.ignite.testframework.MessageOrderLogListener; +import org.apache.ignite.testframework.junits.SystemPropertiesList; +import org.apache.ignite.testframework.junits.WithSystemProperty; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; +import org.junit.Test; + +import static org.apache.ignite.IgniteSystemProperties.IGNITE_LONG_OPERATIONS_DUMP_TIMEOUT; +import static org.apache.ignite.IgniteSystemProperties.IGNITE_LONG_TRANSACTION_TIME_DUMP_THRESHOLD; +import static org.apache.ignite.IgniteSystemProperties.IGNITE_TRANSACTION_TIME_DUMP_SAMPLES_COEFFICIENT; +import static org.apache.ignite.IgniteSystemProperties.IGNITE_TRANSACTION_TIME_DUMP_SAMPLES_PER_SECOND_LIMIT; +import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL; +import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC; +import static org.apache.ignite.internal.processors.cache.transactions.TransactionMetricsAdapter.METRIC_SYSTEM_TIME_HISTOGRAM; +import static org.apache.ignite.internal.processors.cache.transactions.TransactionMetricsAdapter.METRIC_TOTAL_SYSTEM_TIME; +import static org.apache.ignite.internal.processors.cache.transactions.TransactionMetricsAdapter.METRIC_TOTAL_USER_TIME; +import static org.apache.ignite.internal.processors.cache.transactions.TransactionMetricsAdapter.METRIC_USER_TIME_HISTOGRAM; +import static org.apache.ignite.internal.processors.metric.GridMetricManager.TX_METRICS; + +/** + * + */ +@SystemPropertiesList(value = { + @WithSystemProperty(key = IGNITE_LONG_TRANSACTION_TIME_DUMP_THRESHOLD, value = "1000"), + @WithSystemProperty(key = IGNITE_TRANSACTION_TIME_DUMP_SAMPLES_COEFFICIENT, value = "1.0"), + @WithSystemProperty(key = IGNITE_TRANSACTION_TIME_DUMP_SAMPLES_PER_SECOND_LIMIT, value = "5"), + @WithSystemProperty(key = IGNITE_LONG_OPERATIONS_DUMP_TIMEOUT, value = "500") +}) +public class GridTransactionsSystemUserTimeMetricsTest extends GridCommonAbstractTest { + /** */ + private static final String CACHE_NAME = "test"; + + /** */ + private static final String CLIENT = "client"; + + /** */ + private static final String CLIENT_2 = CLIENT + "2"; + + /** */ + private static final long USER_DELAY = 1000; + + /** */ + private static final long SYSTEM_DELAY = 1000; + + /** */ + private static final int TX_COUNT_FOR_LOG_THROTTLING_CHECK = 4; + + /** */ + private static final long LONG_TRAN_TIMEOUT = Math.min(SYSTEM_DELAY, USER_DELAY); + + /** */ + private static final String TRANSACTION_TIME_DUMP_REGEX = ".*?ransaction time dump .*"; + + /** */ + private static final String TRANSACTION_TIME_DUMPS_SKIPPED_REGEX = + "Transaction time dumps skipped because of log throttling: " + TX_COUNT_FOR_LOG_THROTTLING_CHECK / 2; + + /** */ + private LogListener logTxDumpLsnr = new MessageOrderLogListener(TRANSACTION_TIME_DUMP_REGEX); + + /** */ + private final TransactionDumpListener transactionDumpLsnr = new TransactionDumpListener(TRANSACTION_TIME_DUMP_REGEX); + + /** */ + private final TransactionDumpListener transactionDumpsSkippedLsnr = + new TransactionDumpListener(TRANSACTION_TIME_DUMPS_SKIPPED_REGEX); + + /** */ + private final ListeningTestLogger testLog = new ListeningTestLogger(false, log()); + + /** */ + private volatile boolean slowPrepare; + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName); + + testLog.registerListener(logTxDumpLsnr); + testLog.registerListener(transactionDumpLsnr); + testLog.registerListener(transactionDumpsSkippedLsnr); + + cfg.setGridLogger(testLog); + + boolean isClient = igniteInstanceName.contains(CLIENT); + + cfg.setClientMode(isClient); + + if (!isClient) { + CacheConfiguration ccfg = new CacheConfiguration(CACHE_NAME); + + ccfg.setAtomicityMode(TRANSACTIONAL); + ccfg.setBackups(1); + ccfg.setWriteSynchronizationMode(FULL_SYNC); + + cfg.setCacheConfiguration(ccfg); + } + + cfg.setMetricExporterSpi(new JmxMetricExporterSpi()); + + cfg.setCommunicationSpi(new TestCommunicationSpi()); + + return cfg; + } + + /** */ + @Override protected void afterTest() throws Exception { + stopAllGrids(); + + super.afterTest(); + } + + /** */ + @Test + public void testTransactionsSystemUserTime() throws Exception { + Ignite ignite = startGrids(2); + + Ignite client = startGrid(CLIENT); + + IgniteLogger oldLog = GridTestUtils.getFieldValue(IgniteTxAdapter.class, "log"); + + GridTestUtils.setFieldValue(IgniteTxAdapter.class, "log", testLog); + + try { + assertTrue(client.configuration().isClientMode()); + + IgniteCache<Integer, Integer> cache = client.getOrCreateCache(CACHE_NAME); + + cache.put(1, 1); + + Callable<Object> txCallable = () -> { + Integer val = cache.get(1); + + cache.put(1, val + 1); + + return null; + }; + + DynamicMBean tranMBean = metricSet(CLIENT, null, TX_METRICS); + + //slow user + slowPrepare = false; + + doInTransaction(client, () -> { + Integer val = cache.get(1); + + doSleep(USER_DELAY); + + cache.put(1, val + 1); + + return null; + }); + + assertEquals(2, cache.get(1).intValue()); + + assertTrue((Long)tranMBean.getAttribute(METRIC_TOTAL_USER_TIME) >= USER_DELAY); + assertTrue((Long)tranMBean.getAttribute(METRIC_TOTAL_SYSTEM_TIME) < LONG_TRAN_TIMEOUT); + + //slow prepare + slowPrepare = true; + + doInTransaction(client, txCallable); + + assertTrue(logTxDumpLsnr.check()); + + assertEquals(3, cache.get(1).intValue()); + + assertTrue((Long)tranMBean.getAttribute(METRIC_TOTAL_SYSTEM_TIME) >= SYSTEM_DELAY); + + long[] sysTimeHisto = (long[])tranMBean.getAttribute(METRIC_SYSTEM_TIME_HISTOGRAM); + long[] userTimeHisto = (long[])tranMBean.getAttribute(METRIC_USER_TIME_HISTOGRAM); + + assertNotNull(sysTimeHisto); + assertNotNull(userTimeHisto); + + assertTrue(sysTimeHisto != null && sysTimeHisto.length > 0); + assertTrue(userTimeHisto != null && userTimeHisto.length > 0); + + logTxDumpLsnr.reset(); + + //checking settings changing via JMX with second client + Ignite client2 = startGrid(CLIENT_2); + + TransactionsMXBean tmMxBean = getMxBean( + CLIENT, + "Transactions", + TransactionsMXBean.class, + TransactionsMXBeanImpl.class + ); + + tmMxBean.setLongTransactionTimeDumpThreshold(0); + tmMxBean.setTransactionTimeDumpSamplesCoefficient(0.0); + + doInTransaction(client2, txCallable); + + assertFalse(logTxDumpLsnr.check()); + + //testing dumps limit + + doSleep(1000); + + transactionDumpLsnr.reset(); + + transactionDumpsSkippedLsnr.reset(); + + tmMxBean.setTransactionTimeDumpSamplesCoefficient(1.0); + + tmMxBean.setTransactionTimeDumpSamplesPerSecondLimit(TX_COUNT_FOR_LOG_THROTTLING_CHECK / 2); + + slowPrepare = false; + + for (int i = 0; i < TX_COUNT_FOR_LOG_THROTTLING_CHECK; i++) + doInTransaction(client, txCallable); + + assertEquals(TX_COUNT_FOR_LOG_THROTTLING_CHECK / 2, transactionDumpLsnr.value()); + + //testing skipped message in log + + doSleep(1000); + + doInTransaction(client, txCallable); + + assertTrue(transactionDumpsSkippedLsnr.check()); + + U.log(log, sysTimeHisto); + U.log(log, userTimeHisto); + } + finally { + GridTestUtils.setFieldValue(IgniteTxAdapter.class, "log", oldLog); + } + } + + /** + * + */ + private class TestCommunicationSpi extends TcpCommunicationSpi { + /** {@inheritDoc} */ + @Override public void sendMessage(ClusterNode node, Message msg, IgniteInClosure<IgniteException> ackClosure) + throws IgniteSpiException { + if (msg instanceof GridIoMessage) { + Object msg0 = ((GridIoMessage)msg).message(); + + if (slowPrepare && msg0 instanceof GridNearTxPrepareRequest) + doSleep(SYSTEM_DELAY); + } + + super.sendMessage(node, msg, ackClosure); + } + } + + /** + * + */ + private static class TransactionDumpListener extends LogListener { + /** */ + private final AtomicInteger counter = new AtomicInteger(0); + + /** */ + private final String regex; + + /** */ + private TransactionDumpListener(String regex) { + this.regex = regex; + } + + /** {@inheritDoc} */ + @Override public boolean check() { + return value() > 0; + } + + /** {@inheritDoc} */ + @Override public void reset() { + counter.set(0); + } + + /** {@inheritDoc} */ + @Override public void accept(String s) { + if (s.matches(regex)) + counter.incrementAndGet(); + } + + /** */ + int value() { + return counter.get(); + } + } +} diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxDataConsistencyOnCommitFailureTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxDataConsistencyOnCommitFailureTest.java index bed1e9d..a4eace8 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxDataConsistencyOnCommitFailureTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxDataConsistencyOnCommitFailureTest.java @@ -178,7 +178,8 @@ public class TxDataConsistencyOnCommitFailureTest extends GridCommonAbstractTest IgniteTxManager mockTm = Mockito.spy(tm); MockGridNearTxLocal locTx = new MockGridNearTxLocal(ctx, false, false, false, GridIoPolicy.SYSTEM_POOL, - TransactionConcurrency.PESSIMISTIC, TransactionIsolation.REPEATABLE_READ, 0, true, null, 1, null, 0, null); + TransactionConcurrency.PESSIMISTIC, TransactionIsolation.REPEATABLE_READ, 0, true, null, 1, null, 0, null, + null); Mockito.doAnswer(new Answer<GridNearTxLocal>() { @Override public GridNearTxLocal answer(InvocationOnMock invocation) throws Throwable { @@ -222,12 +223,14 @@ public class TxDataConsistencyOnCommitFailureTest extends GridCommonAbstractTest * @param subjId Subj id. * @param taskNameHash Task name hash. * @param lb Label. + * @param txDumpsThrottling Log throttling information. */ public MockGridNearTxLocal(GridCacheSharedContext ctx, boolean implicit, boolean implicitSingle, boolean sys, byte plc, TransactionConcurrency concurrency, TransactionIsolation isolation, long timeout, - boolean storeEnabled, Boolean mvccOp, int txSize, @Nullable UUID subjId, int taskNameHash, @Nullable String lb) { + boolean storeEnabled, Boolean mvccOp, int txSize, @Nullable UUID subjId, int taskNameHash, @Nullable String lb, + IgniteTxManager.TxDumpsThrottling txDumpsThrottling) { super(ctx, implicit, implicitSingle, sys, plc, concurrency, isolation, timeout, storeEnabled, mvccOp, - txSize, subjId, taskNameHash, lb); + txSize, subjId, taskNameHash, lb, txDumpsThrottling); } /** {@inheritDoc} */ diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java index 83afae3..8350d83 100755 --- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java +++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java @@ -20,6 +20,7 @@ package org.apache.ignite.testframework.junits; import java.io.ObjectStreamException; import java.io.Serializable; import java.lang.annotation.Annotation; +import java.lang.management.ManagementFactory; import java.lang.reflect.Constructor; import java.lang.reflect.Field; import java.lang.reflect.InvocationHandler; @@ -44,6 +45,11 @@ import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReference; import javax.cache.configuration.Factory; import javax.cache.configuration.FactoryBuilder; +import javax.management.DynamicMBean; +import javax.management.MBeanServer; +import javax.management.MBeanServerInvocationHandler; +import javax.management.MalformedObjectNameException; +import javax.management.ObjectName; import org.apache.ignite.Ignite; import org.apache.ignite.IgniteCache; import org.apache.ignite.IgniteCheckedException; @@ -2624,4 +2630,29 @@ public abstract class GridAbstractTest extends JUnitAssertAware { } } } + + + /** + * Returns metric set. + * + * @param igniteInstanceName Ignite instance name. + * @param grp Name of the group. + * @param metrics Metrics. + * @return MX bean. + * @throws Exception If failed. + */ + public DynamicMBean metricSet( + String igniteInstanceName, + String grp, + String metrics + ) throws MalformedObjectNameException { + ObjectName mbeanName = U.makeMBeanName(igniteInstanceName, grp, metrics); + + MBeanServer mbeanSrv = ManagementFactory.getPlatformMBeanServer(); + + if (!mbeanSrv.isRegistered(mbeanName)) + throw new IgniteException("MBean not registered."); + + return MBeanServerInvocationHandler.newProxyInstance(mbeanSrv, mbeanName, DynamicMBean.class, false); + } } diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/common/GridCommonAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/common/GridCommonAbstractTest.java index f612156..05b4941 100755 --- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/common/GridCommonAbstractTest.java +++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/common/GridCommonAbstractTest.java @@ -17,6 +17,7 @@ package org.apache.ignite.testframework.junits.common; +import java.lang.management.ManagementFactory; import java.util.ArrayList; import java.util.Arrays; import java.util.BitSet; @@ -37,6 +38,9 @@ import java.util.stream.Collectors; import javax.cache.Cache; import javax.cache.CacheException; import javax.cache.integration.CompletionListener; +import javax.management.MBeanServer; +import javax.management.MBeanServerInvocationHandler; +import javax.management.ObjectName; import javax.net.ssl.HostnameVerifier; import javax.net.ssl.HttpsURLConnection; import javax.net.ssl.SSLSession; @@ -2342,4 +2346,32 @@ public abstract class GridCommonAbstractTest extends GridAbstractTest { cntr0 = cntr.get2(); } } + + /** + * Returns MX bean by specified group name and class. + * + * @param igniteInstanceName Ignite instance name. + * @param grp Name of the group. + * @param cls Bean class. + * @param implCls Bean implementation class. + * @param <T> Type parameter for bean class. + * @param <I> Type parameter for bean implementation class. + * @return MX bean. + * @throws Exception If failed. + */ + protected <T, I> T getMxBean( + String igniteInstanceName, + String grp, + Class<T> cls, + Class<I> implCls + ) throws Exception { + ObjectName mbeanName = U.makeMBeanName(igniteInstanceName, grp, implCls.getSimpleName()); + + MBeanServer mbeanSrv = ManagementFactory.getPlatformMBeanServer(); + + if (!mbeanSrv.isRegistered(mbeanName)) + fail("MBean is not registered: " + mbeanName.getCanonicalName()); + + return MBeanServerInvocationHandler.newProxyInstance(mbeanSrv, mbeanName, cls, true); + } } diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite7.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite7.java index c3fb301..8a598fb 100755 --- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite7.java +++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite7.java @@ -31,6 +31,7 @@ import org.apache.ignite.internal.processors.cache.CacheConfigurationSerializati import org.apache.ignite.internal.processors.cache.CacheDataRegionConfigurationTest; import org.apache.ignite.internal.processors.cache.CacheGroupMetricsTest; import org.apache.ignite.internal.processors.cache.CacheMetricsManageTest; +import org.apache.ignite.internal.processors.cache.GridTransactionsSystemUserTimeMetricsTest; import org.apache.ignite.internal.processors.cache.IgniteDynamicCacheStartFailWithPersistenceTest; import org.apache.ignite.internal.processors.cache.WalModeChangeAdvancedSelfTest; import org.apache.ignite.internal.processors.cache.WalModeChangeCoordinatorNotAffinityNodeSelfTest; @@ -115,6 +116,8 @@ public class IgniteCacheTestSuite7 { GridTestUtils.addTestIfNeeded(suite, TxCrossCacheMapOnInvalidTopologyTest.class, ignoredTests); + GridTestUtils.addTestIfNeeded(suite, GridTransactionsSystemUserTimeMetricsTest.class, ignoredTests); + return suite; } }