import org.apache.ignite.configuration.CacheConfiguration;
import org.apache.ignite.internal.benchmarks.jmh.cache.JmhCacheAbstractBenchmark;
import org.apache.ignite.internal.benchmarks.model.IntValue;
-import org.jsr166.ThreadLocalRandom8;
import org.openjdk.jmh.annotations.Benchmark;
import org.openjdk.jmh.annotations.BenchmarkMode;
import org.openjdk.jmh.annotations.Fork;
/** {@inheritDoc} */
@Override public Duration getExpiryForCreation() {
- boolean generateEvt = ThreadLocalRandom8.current().nextDouble() < rate;
+ boolean generateEvt = ThreadLocalRandom.current().nextDouble() < rate;
return generateEvt ? new Duration(TimeUnit.MILLISECONDS, duration.decrementAndGet()) :
new Duration(TimeUnit.MILLISECONDS, duration.get());
}
/** {@inheritDoc} */
@Override public Duration getExpiryForAccess() {
- boolean generateEvt = ThreadLocalRandom8.current().nextDouble() < rate;
+ boolean generateEvt = ThreadLocalRandom.current().nextDouble() < rate;
return generateEvt ? new Duration(TimeUnit.MILLISECONDS, duration.decrementAndGet()) :
new Duration(TimeUnit.MILLISECONDS, duration.get());
}
/** {@inheritDoc} */
@Override public Duration getExpiryForUpdate() {
- boolean generateEvt = ThreadLocalRandom8.current().nextDouble() < rate;
+ boolean generateEvt = ThreadLocalRandom.current().nextDouble() < rate;
return generateEvt ? new Duration(TimeUnit.MILLISECONDS, duration.decrementAndGet()) :
new Duration(TimeUnit.MILLISECONDS, duration.get());
}
import java.io.IOException;
import java.io.ObjectInput;
import java.io.ObjectOutput;
+import java.util.concurrent.atomic.LongAdder;
import org.apache.ignite.internal.util.typedef.internal.A;
-import org.jsr166.LongAdder8;
/**
* Common functionality implementation for eviction policies with max size/max memory and batch eviction support.
private volatile int batchSize = 1;
/** Memory size occupied by elements in container. */
- protected final LongAdder8 memSize = new LongAdder8();
+ protected final LongAdder memSize = new LongAdder();
/**
* Shrinks backed container to maximum allowed size.
import java.util.Collections;
import java.util.HashSet;
import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.LongAdder;
import java.util.regex.Pattern;
import java.util.regex.PatternSyntaxException;
import org.apache.ignite.IgniteCheckedException;
import org.jetbrains.annotations.Nullable;
import org.jsr166.ConcurrentLinkedDeque8;
import org.jsr166.ConcurrentLinkedDeque8.Node;
-import org.jsr166.LongAdder8;
/**
* IGFS eviction policy which evicts particular blocks.
new ConcurrentLinkedDeque8<>();
/** Current size of all enqueued blocks in bytes. */
- private final LongAdder8 curSize = new LongAdder8();
+ private final LongAdder curSize = new LongAdder();
/**
* Default constructor.
import java.util.Objects;
import java.util.Set;
import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.LongAdder;
import org.apache.ignite.cache.eviction.AbstractEvictionPolicy;
import org.apache.ignite.cache.eviction.EvictableEntry;
import org.apache.ignite.internal.util.GridConcurrentSkipListSet;
import org.apache.ignite.internal.util.typedef.internal.A;
import org.apache.ignite.mxbean.IgniteMBeanAware;
import org.jetbrains.annotations.Nullable;
-import org.jsr166.LongAdder8;
import static java.lang.Math.abs;
import static org.apache.ignite.configuration.CacheConfiguration.DFLT_CACHE_SIZE;
private static final long serialVersionUID = 0L;
/** Size. */
- private final LongAdder8 size = new LongAdder8();
+ private final LongAdder size = new LongAdder();
/**
* @param comp Comparator.
import java.util.Map;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.LongAdder;
import javax.cache.Cache;
import javax.cache.integration.CacheLoaderException;
import javax.cache.integration.CacheWriterException;
import org.apache.ignite.resources.LoggerResource;
import org.apache.ignite.transactions.Transaction;
import org.jetbrains.annotations.Nullable;
-import org.jsr166.LongAdder8;
/**
* {@link CacheStore} implementation backed by JDBC. This implementation
/** Opened connections. */
@GridToStringExclude
- private final LongAdder8 opened = new LongAdder8();
+ private final LongAdder opened = new LongAdder();
/** Closed connections. */
@GridToStringExclude
- private final LongAdder8 closed = new LongAdder8();
+ private final LongAdder closed = new LongAdder();
/** Test mode flag. */
@GridToStringExclude
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicLong;
import java.util.concurrent.atomic.AtomicReference;
+import java.util.concurrent.atomic.LongAdder;
import java.util.concurrent.locks.Lock;
import java.util.concurrent.locks.ReadWriteLock;
import java.util.concurrent.locks.ReentrantReadWriteLock;
import org.jetbrains.annotations.Nullable;
import org.jsr166.ConcurrentHashMap8;
import org.jsr166.ConcurrentLinkedDeque8;
-import org.jsr166.LongAdder8;
import static org.apache.ignite.events.EventType.EVT_NODE_FAILED;
import static org.apache.ignite.events.EventType.EVT_NODE_JOINED;
final AtomicBoolean warmupFinished = new AtomicBoolean();
final AtomicBoolean done = new AtomicBoolean();
final CyclicBarrier bar = new CyclicBarrier(threads + 1);
- final LongAdder8 cnt = new LongAdder8();
+ final LongAdder cnt = new LongAdder();
final long sleepDuration = 5000;
final byte[] payLoad = new byte[payLoadSize];
final Map<UUID, IoTestThreadLocalNodeResults>[] res = new Map[threads];
import java.util.concurrent.Executors;
import java.util.concurrent.Semaphore;
import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.LongAdder;
import java.util.concurrent.locks.ReentrantLock;
import javax.cache.Cache;
import javax.cache.expiry.ExpiryPolicy;
import org.apache.ignite.transactions.TransactionConcurrency;
import org.apache.ignite.transactions.TransactionIsolation;
import org.jetbrains.annotations.Nullable;
-import org.jsr166.LongAdder8;
import static org.apache.ignite.IgniteSystemProperties.IGNITE_CACHE_KEY_VALIDATION_DISABLED;
import static org.apache.ignite.IgniteSystemProperties.IGNITE_CACHE_RETRIES_COUNT;
private boolean mongoMetaCache;
/** Current IGFS data cache size. */
- private LongAdder8 igfsDataCacheSize;
+ private LongAdder igfsDataCacheSize;
/** Asynchronous operations limit semaphore. */
private Semaphore asyncOpsSem;
if (F.eq(ctx.name(), igfsCfg.getDataCacheConfiguration().getName())) {
if (!ctx.isNear()) {
igfsDataCache = true;
- igfsDataCacheSize = new LongAdder8();
+ igfsDataCacheSize = new LongAdder();
}
break;
package org.apache.ignite.internal.processors.cache;
+import java.util.concurrent.atomic.LongAdder;
import org.apache.ignite.IgniteCheckedException;
import org.apache.ignite.IgniteException;
import org.apache.ignite.configuration.CacheConfiguration;
import org.apache.ignite.internal.util.typedef.internal.S;
import org.apache.ignite.internal.util.typedef.internal.U;
import org.jetbrains.annotations.Nullable;
-import org.jsr166.LongAdder8;
/**
* Eagerly removes expired entries from cache when
private static final long serialVersionUID = 0L;
/** Size. */
- private final LongAdder8 size = new LongAdder8();
+ private final LongAdder size = new LongAdder();
/**
* @return Size based on performed operations.
*/
package org.apache.ignite.internal.processors.cache.persistence;
+import java.util.concurrent.atomic.LongAdder;
import org.apache.ignite.DataRegionMetrics;
import org.apache.ignite.configuration.DataRegionConfiguration;
import org.apache.ignite.internal.pagemem.PageMemory;
import org.apache.ignite.internal.util.typedef.internal.U;
import org.apache.ignite.lang.IgniteOutClosure;
import org.jetbrains.annotations.Nullable;
-import org.jsr166.LongAdder8;
/**
*
private final IgniteOutClosure<Float> fillFactorProvider;
/** */
- private final LongAdder8 totalAllocatedPages = new LongAdder8();
+ private final LongAdder totalAllocatedPages = new LongAdder();
/**
* Counter for number of pages occupied by large entries (one entry is larger than one page).
*/
- private final LongAdder8 largeEntriesPages = new LongAdder8();
+ private final LongAdder largeEntriesPages = new LongAdder();
/** Counter for number of dirty pages. */
- private LongAdder8 dirtyPages = new LongAdder8();
+ private LongAdder dirtyPages = new LongAdder();
/** */
private volatile boolean metricsEnabled;
import java.io.IOException;
import java.io.ObjectInput;
import java.io.ObjectOutput;
+import java.util.concurrent.atomic.LongAdder;
import org.apache.ignite.cache.query.QueryMetrics;
import org.apache.ignite.internal.util.GridAtomicLong;
import org.apache.ignite.internal.util.typedef.internal.S;
-import org.jsr166.LongAdder8;
/**
* Adapter for {@link QueryMetrics}.
private final GridAtomicLong maxTime = new GridAtomicLong();
/** Sum of execution time for all completed queries. */
- private final LongAdder8 sumTime = new LongAdder8();
+ private final LongAdder sumTime = new LongAdder();
/** Average time of execution.
* If doesn't equal zero then this metrics set is copy from remote node and doesn't actually update.
private double avgTime;
/** Number of executions. */
- private final LongAdder8 execs = new LongAdder8();
+ private final LongAdder execs = new LongAdder();
/** Number of completed executions. */
- private final LongAdder8 completed = new LongAdder8();
+ private final LongAdder completed = new LongAdder();
/** Number of fails. */
- private final LongAdder8 fails = new LongAdder8();
+ private final LongAdder fails = new LongAdder();
/** {@inheritDoc} */
@Override public long minimumTime() {
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicLong;
import java.util.concurrent.atomic.AtomicReference;
+import java.util.concurrent.atomic.LongAdder;
import javax.cache.CacheException;
import javax.cache.expiry.ExpiryPolicy;
import org.apache.ignite.Ignite;
import org.apache.ignite.stream.StreamReceiver;
import org.jetbrains.annotations.Nullable;
import org.jsr166.ConcurrentHashMap8;
-import org.jsr166.LongAdder8;
import static org.apache.ignite.events.EventType.EVT_NODE_FAILED;
import static org.apache.ignite.events.EventType.EVT_NODE_LEFT;
private volatile Throwable cancellationReason = null;
/** Fail counter. */
- private final LongAdder8 failCntr = new LongAdder8();
+ private final LongAdder failCntr = new LongAdder();
/** Active futures of this data loader. */
@GridToStringInclude
package org.apache.ignite.internal.processors.igfs;
+import java.util.concurrent.atomic.LongAdder;
import org.apache.ignite.internal.util.typedef.F;
import org.apache.ignite.lang.IgniteBiTuple;
-import org.jsr166.LongAdder8;
/**
* Value object holding all local IGFS metrics which cannot be determined using file system traversal.
*/
public class IgfsLocalMetrics {
/** Block reads. First value - total reads, second value - reads delegated to the secondary file system. */
- private volatile IgniteBiTuple<LongAdder8, LongAdder8> blocksRead;
+ private volatile IgniteBiTuple<LongAdder, LongAdder> blocksRead;
/** Block writes. First value - total writes, second value - writes delegated to the secondary file system. */
- private volatile IgniteBiTuple<LongAdder8, LongAdder8> blocksWritten;
+ private volatile IgniteBiTuple<LongAdder, LongAdder> blocksWritten;
/** Byte reads. First value - total bytes read, second value - consumed time. */
- private volatile IgniteBiTuple<LongAdder8, LongAdder8> bytesRead;
+ private volatile IgniteBiTuple<LongAdder, LongAdder> bytesRead;
/** Byte writes. First value - total bytes written, second value - consumed time. */
- private volatile IgniteBiTuple<LongAdder8, LongAdder8> bytesWritten;
+ private volatile IgniteBiTuple<LongAdder, LongAdder> bytesWritten;
/** Number of files opened for read. */
- private final LongAdder8 filesOpenedForRead = new LongAdder8();
+ private final LongAdder filesOpenedForRead = new LongAdder();
/** Number of files opened for write. */
- private final LongAdder8 filesOpenedForWrite = new LongAdder8();
+ private final LongAdder filesOpenedForWrite = new LongAdder();
/**
* Constructor.
* @param readTime Read time.
*/
void addReadBytesTime(long readBytes, long readTime) {
- IgniteBiTuple<LongAdder8, LongAdder8> bytesRead0 = bytesRead;
+ IgniteBiTuple<LongAdder, LongAdder> bytesRead0 = bytesRead;
bytesRead0.get1().add(readBytes);
bytesRead0.get2().add(readTime);
* @param writeTime Write time.
*/
void addWrittenBytesTime(long writtenBytes, long writeTime) {
- IgniteBiTuple<LongAdder8, LongAdder8> bytesWritten0 = bytesWritten;
+ IgniteBiTuple<LongAdder, LongAdder> bytesWritten0 = bytesWritten;
bytesWritten0.get1().add(writtenBytes);
bytesWritten0.get2().add(writeTime);
* @param secondary Number of blocks read form secondary FS.
*/
void addReadBlocks(int total, int secondary) {
- IgniteBiTuple<LongAdder8, LongAdder8> blocksRead0 = blocksRead;
+ IgniteBiTuple<LongAdder, LongAdder> blocksRead0 = blocksRead;
blocksRead0.get1().add(total);
blocksRead0.get2().add(secondary);
* @param secondary Number of blocks written to secondary FS.
*/
void addWriteBlocks(int total, int secondary) {
- IgniteBiTuple<LongAdder8, LongAdder8> blocksWritten0 = blocksWritten;
+ IgniteBiTuple<LongAdder, LongAdder> blocksWritten0 = blocksWritten;
blocksWritten0.get1().add(total);
blocksWritten0.get2().add(secondary);
* Reset summary counters.
*/
void reset() {
- blocksRead = F.t(new LongAdder8(), new LongAdder8());
- blocksWritten = F.t(new LongAdder8(), new LongAdder8());
- bytesRead = F.t(new LongAdder8(), new LongAdder8());
- bytesWritten = F.t(new LongAdder8(), new LongAdder8());
+ blocksRead = F.t(new LongAdder(), new LongAdder());
+ blocksWritten = F.t(new LongAdder(), new LongAdder());
+ bytesRead = F.t(new LongAdder(), new LongAdder());
+ bytesWritten = F.t(new LongAdder(), new LongAdder());
}
}
\ No newline at end of file
import java.util.concurrent.Executor;
import java.util.concurrent.RejectedExecutionException;
import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.LongAdder;
import java.util.concurrent.locks.Condition;
import java.util.concurrent.locks.Lock;
import java.util.concurrent.locks.ReentrantLock;
import org.jetbrains.annotations.Nullable;
import org.jsr166.ConcurrentHashMap8;
import org.jsr166.ConcurrentLinkedHashMap;
-import org.jsr166.LongAdder8;
import static java.util.concurrent.TimeUnit.MILLISECONDS;
import static org.apache.ignite.IgniteSystemProperties.IGNITE_JOBS_HISTORY_SIZE;
private final GridLocalEventListener discoLsnr;
/** Needed for statistics. */
- private final LongAdder8 canceledJobsCnt = new LongAdder8();
+ private final LongAdder canceledJobsCnt = new LongAdder();
/** Needed for statistics. */
- private final LongAdder8 finishedJobsCnt = new LongAdder8();
+ private final LongAdder finishedJobsCnt = new LongAdder();
/** Needed for statistics. */
- private final LongAdder8 startedJobsCnt = new LongAdder8();
+ private final LongAdder startedJobsCnt = new LongAdder();
/** Needed for statistics. */
- private final LongAdder8 rejectedJobsCnt = new LongAdder8();
+ private final LongAdder rejectedJobsCnt = new LongAdder();
/** Total job execution time (unaccounted for in metrics). */
- private final LongAdder8 finishedJobsTime = new LongAdder8();
+ private final LongAdder finishedJobsTime = new LongAdder();
/** Maximum job execution time for finished jobs. */
private final GridAtomicLong maxFinishedJobsTime = new GridAtomicLong();
package org.apache.ignite.internal.processors.jobmetrics;
+import java.util.concurrent.ThreadLocalRandom;
import java.util.concurrent.atomic.AtomicBoolean;
import org.apache.ignite.IgniteCheckedException;
import org.apache.ignite.internal.GridKernalContext;
import org.apache.ignite.internal.util.typedef.X;
import org.apache.ignite.internal.util.typedef.internal.U;
import org.apache.ignite.lang.IgniteReducer;
-import org.jsr166.ThreadLocalRandom8;
import static org.apache.ignite.IgniteSystemProperties.IGNITE_JOBS_METRICS_CONCURRENCY_LEVEL;
InternalMetrics m = this.metrics;
- m.snapshotsQueues[ThreadLocalRandom8.current().nextInt(m.snapshotsQueues.length)].add(metrics);
+ m.snapshotsQueues[ThreadLocalRandom.current().nextInt(m.snapshotsQueues.length)].add(metrics);
// Handle current and total idle times.
long idleTimer0 = idleTimer;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.RejectedExecutionException;
import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.LongAdder;
import org.apache.ignite.IgniteCheckedException;
import org.apache.ignite.IgniteSystemProperties;
import org.apache.ignite.configuration.ConnectorConfiguration;
import org.apache.ignite.internal.processors.rest.client.message.GridClientTaskResultBean;
import org.apache.ignite.internal.processors.rest.handlers.GridRestCommandHandler;
import org.apache.ignite.internal.processors.rest.handlers.cache.GridCacheCommandHandler;
+import org.apache.ignite.internal.processors.rest.handlers.cluster.GridChangeStateCommandHandler;
import org.apache.ignite.internal.processors.rest.handlers.datastructures.DataStructuresCommandHandler;
import org.apache.ignite.internal.processors.rest.handlers.log.GridLogCommandHandler;
import org.apache.ignite.internal.processors.rest.handlers.query.QueryCommandHandler;
-import org.apache.ignite.internal.processors.rest.handlers.cluster.GridChangeStateCommandHandler;
import org.apache.ignite.internal.processors.rest.handlers.task.GridTaskCommandHandler;
import org.apache.ignite.internal.processors.rest.handlers.top.GridTopologyCommandHandler;
import org.apache.ignite.internal.processors.rest.handlers.version.GridVersionCommandHandler;
import org.apache.ignite.plugin.security.SecurityException;
import org.apache.ignite.plugin.security.SecurityPermission;
import org.apache.ignite.thread.IgniteThread;
-import org.jsr166.LongAdder8;
import static org.apache.ignite.IgniteSystemProperties.IGNITE_REST_START_ON_CLIENT;
import static org.apache.ignite.internal.processors.rest.GridRestResponse.STATUS_AUTH_FAILED;
private final GridSpinReadWriteLock busyLock = new GridSpinReadWriteLock();
/** Workers count. */
- private final LongAdder8 workersCnt = new LongAdder8();
+ private final LongAdder workersCnt = new LongAdder();
/** ClientId-SessionId map. */
private final ConcurrentMap<UUID, UUID> clientId2SesId = new ConcurrentHashMap<>();
import java.util.List;
import java.util.Map;
import java.util.UUID;
+import java.util.concurrent.ThreadLocalRandom;
import java.util.concurrent.atomic.AtomicReference;
import org.apache.ignite.Ignite;
import org.apache.ignite.IgniteCheckedException;
import org.apache.ignite.lang.IgniteProductVersion;
import org.apache.ignite.resources.IgniteInstanceResource;
import org.apache.ignite.services.Service;
-import org.jsr166.ThreadLocalRandom8;
import static org.apache.ignite.internal.processors.task.GridTaskThreadContextKey.TC_IO_POLICY;
// Optimization if projection is the whole grid.
if (prj.predicate() == F.<ClusterNode>alwaysTrue()) {
- int idx = ThreadLocalRandom8.current().nextInt(snapshot.size());
+ int idx = ThreadLocalRandom.current().nextInt(snapshot.size());
int i = 0;
if (nodeList.isEmpty())
return null;
- int idx = ThreadLocalRandom8.current().nextInt(nodeList.size());
+ int idx = ThreadLocalRandom.current().nextInt(nodeList.size());
return nodeList.get(idx);
}
import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.RejectedExecutionException;
+import java.util.concurrent.atomic.LongAdder;
import org.apache.ignite.IgniteCheckedException;
import org.apache.ignite.IgniteException;
import org.apache.ignite.cluster.ClusterNode;
import org.apache.ignite.marshaller.Marshaller;
import org.apache.ignite.plugin.security.SecurityPermission;
import org.jetbrains.annotations.Nullable;
-import org.jsr166.LongAdder8;
import static org.apache.ignite.events.EventType.EVT_NODE_FAILED;
import static org.apache.ignite.events.EventType.EVT_NODE_LEFT;
private final GridLocalEventListener discoLsnr;
/** Total executed tasks. */
- private final LongAdder8 execTasks = new LongAdder8();
+ private final LongAdder execTasks = new LongAdder();
/** */
private final ThreadLocal<Map<GridTaskThreadContextKey, Object>> thCtx = new ThreadLocal<>();
import java.util.UUID;
import java.util.concurrent.Callable;
import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ThreadLocalRandom;
import java.util.concurrent.atomic.AtomicInteger;
import javax.cache.Cache;
import org.apache.ignite.IgniteCheckedException;
import org.apache.ignite.lang.IgniteReducer;
import org.jetbrains.annotations.Nullable;
import org.jsr166.ConcurrentLinkedDeque8;
-import org.jsr166.ThreadLocalRandom8;
/**
* Contains factory and utility methods for {@code closures}, {@code predicates}, and {@code tuples}.
public static <T> T rand(Collection<? extends T> c) {
A.notNull(c, "c");
- int n = ThreadLocalRandom8.current().nextInt(c.size());
+ int n = ThreadLocalRandom.current().nextInt(c.size());
int i = 0;
public static <T> T rand(List<T> l) {
A.notNull(l, "l");
- return l.get(ThreadLocalRandom8.current().nextInt(l.size()));
+ return l.get(ThreadLocalRandom.current().nextInt(l.size()));
}
/**
public static <T> T rand(T... c) {
A.notNull(c, "c");
- return c[ThreadLocalRandom8.current().nextInt(c.length)];
+ return c[ThreadLocalRandom.current().nextInt(c.length)];
}
/**
import java.util.LinkedList;
import java.util.NoSuchElementException;
import java.util.Queue;
+import java.util.concurrent.atomic.LongAdder;
import java.util.concurrent.locks.ReadWriteLock;
import java.util.concurrent.locks.ReentrantReadWriteLock;
import org.apache.ignite.IgniteCheckedException;
import org.apache.ignite.lang.IgniteBiPredicate;
import org.apache.ignite.lang.IgniteBiTuple;
import org.jetbrains.annotations.Nullable;
-import org.jsr166.LongAdder8;
import static org.apache.ignite.internal.util.offheap.GridOffHeapEvent.REHASH;
private final GridUnsafeLru lru;
/** Total entry count. */
- private final LongAdder8 totalCnt;
+ private final LongAdder totalCnt;
/** Event listener. */
private GridOffHeapEventListener evtLsnr;
if (lru != null)
this.evictLsnr = evictLsnr;
- totalCnt = new LongAdder8();
+ totalCnt = new LongAdder();
// Find power-of-two sizes best matching arguments
int shift = 0;
* @param lruPoller LRU poller.
*/
@SuppressWarnings("unchecked")
- GridUnsafeMap(int part, int concurrency, float load, long initCap, LongAdder8 totalCnt, GridUnsafeMemory mem,
+ GridUnsafeMap(int part, int concurrency, float load, long initCap, LongAdder totalCnt, GridUnsafeMemory mem,
GridUnsafeLru lru, @Nullable GridOffHeapEvictListener evictLsnr, GridUnsafeLruPoller lruPoller) {
this.part = part;
this.concurrency = concurrency > MAX_CONCURRENCY ? MAX_CONCURRENCY : concurrency;
import java.util.NoSuchElementException;
import java.util.Set;
+import java.util.concurrent.atomic.LongAdder;
import org.apache.ignite.IgniteCheckedException;
import org.apache.ignite.internal.util.GridCloseableIteratorAdapter;
import org.apache.ignite.internal.util.lang.GridCloseableIterator;
import org.apache.ignite.lang.IgniteBiPredicate;
import org.apache.ignite.lang.IgniteBiTuple;
import org.jetbrains.annotations.Nullable;
-import org.jsr166.LongAdder8;
/**
* Off-heap map based on {@code Unsafe} implementation.
private final int parts;
/** */
- private final LongAdder8 totalCnt = new LongAdder8();
+ private final LongAdder totalCnt = new LongAdder();
/**
* @param parts Partitions.
import java.util.concurrent.Callable;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.atomic.LongAdder;
import org.apache.ignite.internal.managers.communication.GridIoMessage;
import org.apache.ignite.internal.util.nio.GridNioMetricsListener;
import org.apache.ignite.internal.util.typedef.F;
import org.apache.ignite.internal.util.typedef.internal.U;
import org.apache.ignite.plugin.extensions.communication.Message;
-import org.jsr166.LongAdder8;
/**
* Statistics for {@link org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi}.
*/
public class TcpCommunicationMetricsListener implements GridNioMetricsListener{
/** Received messages count. */
- private final LongAdder8 rcvdMsgsCnt = new LongAdder8();
+ private final LongAdder rcvdMsgsCnt = new LongAdder();
/** Sent messages count.*/
- private final LongAdder8 sentMsgsCnt = new LongAdder8();
+ private final LongAdder sentMsgsCnt = new LongAdder();
/** Received bytes count. */
- private final LongAdder8 rcvdBytesCnt = new LongAdder8();
+ private final LongAdder rcvdBytesCnt = new LongAdder();
/** Sent bytes count.*/
- private final LongAdder8 sentBytesCnt = new LongAdder8();
+ private final LongAdder sentBytesCnt = new LongAdder();
/** Counter factory. */
- private static final Callable<LongAdder8> LONG_ADDER_FACTORY = new Callable<LongAdder8>() {
- @Override public LongAdder8 call() {
- return new LongAdder8();
+ private static final Callable<LongAdder> LONG_ADDER_FACTORY = new Callable<LongAdder>() {
+ @Override public LongAdder call() {
+ return new LongAdder();
}
};
/** Received messages count grouped by message type. */
- private final ConcurrentMap<String, LongAdder8> rcvdMsgsCntByType = new ConcurrentHashMap<>();
+ private final ConcurrentMap<String, LongAdder> rcvdMsgsCntByType = new ConcurrentHashMap<>();
/** Received messages count grouped by sender. */
- private final ConcurrentMap<UUID, LongAdder8> rcvdMsgsCntByNode = new ConcurrentHashMap<>();
+ private final ConcurrentMap<UUID, LongAdder> rcvdMsgsCntByNode = new ConcurrentHashMap<>();
/** Sent messages count grouped by message type. */
- private final ConcurrentMap<String, LongAdder8> sentMsgsCntByType = new ConcurrentHashMap<>();
+ private final ConcurrentMap<String, LongAdder> sentMsgsCntByType = new ConcurrentHashMap<>();
/** Sent messages count grouped by receiver. */
- private final ConcurrentMap<UUID, LongAdder8> sentMsgsCntByNode = new ConcurrentHashMap<>();
+ private final ConcurrentMap<UUID, LongAdder> sentMsgsCntByNode = new ConcurrentHashMap<>();
/** {@inheritDoc} */
@Override public void onBytesSent(int bytesCnt) {
if (msg instanceof GridIoMessage)
msg = ((GridIoMessage)msg).message();
- LongAdder8 cntByType = F.addIfAbsent(sentMsgsCntByType, msg.getClass().getSimpleName(), LONG_ADDER_FACTORY);
- LongAdder8 cntByNode = F.addIfAbsent(sentMsgsCntByNode, nodeId, LONG_ADDER_FACTORY);
+ LongAdder cntByType = F.addIfAbsent(sentMsgsCntByType, msg.getClass().getSimpleName(), LONG_ADDER_FACTORY);
+ LongAdder cntByNode = F.addIfAbsent(sentMsgsCntByNode, nodeId, LONG_ADDER_FACTORY);
cntByType.increment();
cntByNode.increment();
if (msg instanceof GridIoMessage)
msg = ((GridIoMessage)msg).message();
- LongAdder8 cntByType = F.addIfAbsent(rcvdMsgsCntByType, msg.getClass().getSimpleName(), LONG_ADDER_FACTORY);
- LongAdder8 cntByNode = F.addIfAbsent(rcvdMsgsCntByNode, nodeId, LONG_ADDER_FACTORY);
+ LongAdder cntByType = F.addIfAbsent(rcvdMsgsCntByType, msg.getClass().getSimpleName(), LONG_ADDER_FACTORY);
+ LongAdder cntByNode = F.addIfAbsent(rcvdMsgsCntByNode, nodeId, LONG_ADDER_FACTORY);
cntByType.increment();
cntByNode.increment();
* @param srcStat Internal statistics representation.
* @return Result map.
*/
- private <T> Map<T, Long> convertStatistics(Map<T, LongAdder8> srcStat) {
+ private <T> Map<T, Long> convertStatistics(Map<T, LongAdder> srcStat) {
Map<T, Long> destStat = U.newHashMap(srcStat.size());
- for (Map.Entry<T, LongAdder8> entry : srcStat.entrySet())
+ for (Map.Entry<T, LongAdder> entry : srcStat.entrySet())
destStat.put(entry.getKey(), entry.getValue().longValue());
return destStat;
import java.util.NoSuchElementException;
import java.util.Set;
import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.atomic.LongAdder;
import java.util.concurrent.locks.AbstractQueuedSynchronizer;
/**
* same mapping value.
*
* <p>A ConcurrentHashMapV8 can be used as scalable frequency map (a
- * form of histogram or multiset) by using {@link LongAdder8} values
+ * form of histogram or multiset) by using {@link LongAdder} values
* and initializing via {@link #computeIfAbsent}. For example, to add
* a count to a {@code ConcurrentHashMapV8<String,LongAdder8> freqs}, you
* can use {@code freqs.computeIfAbsent(k -> new
/**
* The counter maintaining number of elements.
*/
- private transient final LongAdder8 counter;
+ private transient final LongAdder counter;
/**
* Table initialization and resizing control. When negative, the
*/
final void tryAwaitLock(Node[] tab, int i) {
if (tab != null && i >= 0 && i < tab.length) { // sanity check
- int r = ThreadLocalRandom8.current().nextInt(); // randomize spins
+ int r = ThreadLocalRandom.current().nextInt(); // randomize spins
int spins = MAX_SPINS, h;
while (tabAt(tab, i) == this && ((h = hash) & LOCKED) != 0) {
if (spins >= 0) {
* Creates a new, empty map with the default initial table size (16).
*/
public ConcurrentHashMap8() {
- this.counter = new LongAdder8();
+ this.counter = new LongAdder();
}
/**
int cap = ((initialCapacity >= (MAXIMUM_CAPACITY >>> 1)) ?
MAXIMUM_CAPACITY :
tableSizeFor(initialCapacity + (initialCapacity >>> 1) + 1));
- this.counter = new LongAdder8();
+ this.counter = new LongAdder();
this.sizeCtl = cap;
}
* @param m the map
*/
public ConcurrentHashMap8(Map<? extends K, ? extends V> m) {
- this.counter = new LongAdder8();
+ this.counter = new LongAdder();
this.sizeCtl = DEFAULT_CAPACITY;
internalPutAll(m);
}
long size = (long)(1.0 + (long)initialCapacity / loadFactor);
int cap = (size >= (long)MAXIMUM_CAPACITY) ?
MAXIMUM_CAPACITY : tableSizeFor((int)size);
- this.counter = new LongAdder8();
+ this.counter = new LongAdder();
this.sizeCtl = cap;
}
s.defaultReadObject();
this.segments = null; // unneeded
// initialize transient final field
- UNSAFE.putObjectVolatile(this, counterOffset, new LongAdder8());
+ UNSAFE.putObjectVolatile(this, counterOffset, new LongAdder());
// Create all nodes, then place in table once size is known
long size = 0L;
import java.util.Iterator;
import java.util.NoSuchElementException;
import java.util.Queue;
+import java.util.concurrent.atomic.LongAdder;
import sun.misc.Unsafe;
private transient volatile Node<E> tail;
/** */
- private final LongAdder8 size = new LongAdder8();
+ private final LongAdder size = new LongAdder();
private static final Node<Object> PREV_TERMINATOR, NEXT_TERMINATOR;
import java.util.Queue;
import java.util.Set;
import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.atomic.LongAdder;
import java.util.concurrent.locks.ReentrantReadWriteLock;
import org.apache.ignite.internal.util.tostring.GridToStringExclude;
import org.apache.ignite.internal.util.typedef.internal.S;
private final ConcurrentLinkedDeque8<HashEntry<K, V>> entryQ;
/** Atomic variable containing map size. */
- private final LongAdder8 size = new LongAdder8();
+ private final LongAdder size = new LongAdder();
/** */
- private final LongAdder8 modCnt = new LongAdder8();
+ private final LongAdder modCnt = new LongAdder();
/** */
private final int maxCap;
+++ /dev/null
-/*
- * Written by Doug Lea with assistance from members of JCP JSR-166
- * Expert Group and released to the public domain, as explained at
- * http://creativecommons.org/publicdomain/zero/1.0/
- */
-
-/*
- * The latest version of the file corresponds to the following CVS commit:
- * http://gee.cs.oswego.edu/cgi-bin/viewcvs.cgi/jsr166/src/main/java/util/concurrent/atomic/LongAdder.java?pathrev=1.3
- *
- * The later versions are based on updated Striped64 that uses java.util.function package which is unavailable in JDK 7.
- * Thus they can't be imported.
- */
-
-package org.jsr166;
-
-import java.io.IOException;
-import java.io.ObjectInputStream;
-import java.io.Serializable;
-import java.util.concurrent.atomic.AtomicLong;
-
-/**
- * One or more variables that together maintain an initially zero
- * {@code long} sum. When updates (method {@link #add}) are contended
- * across threads, the set of variables may grow dynamically to reduce
- * contention. Method {@link #sum} (or, equivalently, {@link
- * #longValue}) returns the current total combined across the
- * variables maintaining the sum.
- *
- * <p>This class is usually preferable to {@link AtomicLong} when
- * multiple threads update a common sum that is used for purposes such
- * as collecting statistics, not for fine-grained synchronization
- * control. Under low update contention, the two classes have similar
- * characteristics. But under high contention, expected throughput of
- * this class is significantly higher, at the expense of higher space
- * consumption.
- *
- * <p>This class extends {@link Number}, but does <em>not</em> define
- * methods such as {@code hashCode} and {@code compareTo} because
- * instances are expected to be mutated, and so are not useful as
- * collection keys.
- *
- * <p><em>jsr166e note: This class is targeted to be placed in
- * java.util.concurrent.atomic.</em>
- *
- * @since 1.8
- * @author Doug Lea
- */
-@SuppressWarnings("ALL")
-public class LongAdder8 extends Striped64_8 implements Serializable {
- private static final long serialVersionUID = 7249069246863182397L;
-
- /**
- * Version of plus for use in retryUpdate
- */
- final long fn(long v, long x) { return v + x; }
-
- /**
- * Creates a new adder with initial sum of zero.
- */
- public LongAdder8() {
- }
-
- /**
- * Adds the given value.
- *
- * @param x the value to add
- */
- public void add(long x) {
- Cell[] as; long b, v; HashCode hc; Cell a; int n;
- if ((as = cells) != null || !casBase(b = base, b + x)) {
- boolean uncontended = true;
- int h = (hc = threadHashCode.get()).code;
- if (as == null || (n = as.length) < 1 ||
- (a = as[(n - 1) & h]) == null ||
- !(uncontended = a.cas(v = a.value, v + x)))
- retryUpdate(x, hc, uncontended);
- }
- }
-
- /**
- * Equivalent to {@code add(1)}.
- */
- public void increment() {
- add(1L);
- }
-
- /**
- * Equivalent to {@code add(-1)}.
- */
- public void decrement() {
- add(-1L);
- }
-
- /**
- * Returns the current sum. The returned value is <em>NOT</em> an
- * atomic snapshot: Invocation in the absence of concurrent
- * updates returns an accurate result, but concurrent updates that
- * occur while the sum is being calculated might not be
- * incorporated.
- *
- * @return the sum
- */
- public long sum() {
- long sum = base;
- Cell[] as = cells;
- if (as != null) {
- int n = as.length;
- for (int i = 0; i < n; ++i) {
- Cell a = as[i];
- if (a != null)
- sum += a.value;
- }
- }
- return sum;
- }
-
- /**
- * Resets variables maintaining the sum to zero. This method may
- * be a useful alternative to creating a new adder, but is only
- * effective if there are no concurrent updates. Because this
- * method is intrinsically racy, it should only be used when it is
- * known that no threads are concurrently updating.
- */
- public void reset() {
- internalReset(0L);
- }
-
- /**
- * Equivalent in effect to {@link #sum} followed by {@link
- * #reset}. This method may apply for example during quiescent
- * points between multithreaded computations. If there are
- * updates concurrent with this method, the returned value is
- * <em>not</em> guaranteed to be the final value occurring before
- * the reset.
- *
- * @return the sum
- */
- public long sumThenReset() {
- long sum = base;
- Cell[] as = cells;
- base = 0L;
- if (as != null) {
- int n = as.length;
- for (int i = 0; i < n; ++i) {
- Cell a = as[i];
- if (a != null) {
- sum += a.value;
- a.value = 0L;
- }
- }
- }
- return sum;
- }
-
- /**
- * Returns the String representation of the {@link #sum}.
- * @return the String representation of the {@link #sum}
- */
- public String toString() {
- return Long.toString(sum());
- }
-
- /**
- * Equivalent to {@link #sum}.
- *
- * @return the sum
- */
- public long longValue() {
- return sum();
- }
-
- /**
- * Returns the {@link #sum} as an {@code int} after a narrowing
- * primitive conversion.
- */
- public int intValue() {
- return (int)sum();
- }
-
- /**
- * Returns the {@link #sum} as a {@code float}
- * after a widening primitive conversion.
- */
- public float floatValue() {
- return (float)sum();
- }
-
- /**
- * Returns the {@link #sum} as a {@code double} after a widening
- * primitive conversion.
- */
- public double doubleValue() {
- return (double)sum();
- }
-
- private void writeObject(java.io.ObjectOutputStream s)
- throws java.io.IOException {
- s.defaultWriteObject();
- s.writeLong(sum());
- }
-
- private void readObject(ObjectInputStream s)
- throws IOException, ClassNotFoundException {
- s.defaultReadObject();
- busy = 0;
- cells = null;
- base = s.readLong();
- }
-
-}
\ No newline at end of file
+++ /dev/null
-/*
- * Written by Doug Lea with assistance from members of JCP JSR-166
- * Expert Group and released to the public domain, as explained at
- * http://creativecommons.org/publicdomain/zero/1.0/
- */
-
-/*
- * The latest version of the file corresponds to the following CVS commit:
- * http://gee.cs.oswego.edu/cgi-bin/viewcvs.cgi/jsr166/src/main/java/util/concurrent/atomic/Striped64.java?pathrev=1.1
- *
- * The later versions use classes from java.util.function package that are unavailable in JDK 7.
- * Thus they can't be imported.
- */
-
-package org.jsr166;
-
-import java.util.Random;
-
-/**
- * A package-local class holding common representation and mechanics
- * for classes supporting dynamic striping on 64bit values. The class
- * extends Number so that concrete subclasses must publicly do so.
- */
-@SuppressWarnings("ALL")
-abstract class Striped64_8 extends Number {
- /*
- * This class maintains a lazily-initialized table of atomically
- * updated variables, plus an extra "base" field. The table size
- * is a power of two. Indexing uses masked per-thread hash codes.
- * Nearly all declarations in this class are package-private,
- * accessed directly by subclasses.
- *
- * Table entries are of class Cell; a variant of AtomicLong padded
- * to reduce cache contention on most processors. Padding is
- * overkill for most Atomics because they are usually irregularly
- * scattered in memory and thus don't interfere much with each
- * other. But Atomic objects residing in arrays will tend to be
- * placed adjacent to each other, and so will most often share
- * cache lines (with a huge negative performance impact) without
- * this precaution.
- *
- * In part because Cells are relatively large, we avoid creating
- * them until they are needed. When there is no contention, all
- * updates are made to the base field. Upon first contention (a
- * failed CAS on base update), the table is initialized to size 2.
- * The table size is doubled upon further contention until
- * reaching the nearest power of two greater than or equal to the
- * number of CPUS. Table slots remain empty (null) until they are
- * needed.
- *
- * A single spinlock ("busy") is used for initializing and
- * resizing the table, as well as populating slots with new Cells.
- * There is no need for a blocking lock: When the lock is not
- * available, threads try other slots (or the base). During these
- * retries, there is increased contention and reduced locality,
- * which is still better than alternatives.
- *
- * Per-thread hash codes are initialized to random values.
- * Contention and/or table collisions are indicated by failed
- * CASes when performing an update operation (see method
- * retryUpdate). Upon a collision, if the table size is less than
- * the capacity, it is doubled in size unless some other thread
- * holds the lock. If a hashed slot is empty, and lock is
- * available, a new Cell is created. Otherwise, if the slot
- * exists, a CAS is tried. Retries proceed by "double hashing",
- * using a secondary hash (Marsaglia XorShift) to try to find a
- * free slot.
- *
- * The table size is capped because, when there are more threads
- * than CPUs, supposing that each thread were bound to a CPU,
- * there would exist a perfect hash function mapping threads to
- * slots that eliminates collisions. When we reach capacity, we
- * search for this mapping by randomly varying the hash codes of
- * colliding threads. Because search is random, and collisions
- * only become known via CAS failures, convergence can be slow,
- * and because threads are typically not bound to CPUS forever,
- * may not occur at all. However, despite these limitations,
- * observed contention rates are typically low in these cases.
- *
- * It is possible for a Cell to become unused when threads that
- * once hashed to it terminate, as well as in the case where
- * doubling the table causes no thread to hash to it under
- * expanded mask. We do not try to detect or remove such cells,
- * under the assumption that for long-running instances, observed
- * contention levels will recur, so the cells will eventually be
- * needed again; and for short-lived ones, it does not matter.
- */
-
- /**
- * Padded variant of AtomicLong supporting only raw accesses plus CAS.
- * The value field is placed between pads, hoping that the JVM doesn't
- * reorder them.
- *
- * JVM intrinsics note: It would be possible to use a release-only
- * form of CAS here, if it were provided.
- */
- static final class Cell {
- volatile long p0, p1, p2, p3, p4, p5, p6;
- volatile long value;
- volatile long q0, q1, q2, q3, q4, q5, q6;
- Cell(long x) { value = x; }
-
- final boolean cas(long cmp, long val) {
- return UNSAFE.compareAndSwapLong(this, valueOffset, cmp, val);
- }
-
- // Unsafe mechanics
- private static final sun.misc.Unsafe UNSAFE;
- private static final long valueOffset;
- static {
- try {
- UNSAFE = getUnsafe();
- Class<?> ak = Cell.class;
- valueOffset = UNSAFE.objectFieldOffset
- (ak.getDeclaredField("value"));
- } catch (Exception e) {
- throw new Error(e);
- }
- }
-
- }
-
- /**
- * Holder for the thread-local hash code. The code is initially
- * random, but may be set to a different value upon collisions.
- */
- static final class HashCode {
- static final Random rng = new Random();
- int code;
- HashCode() {
- int h = rng.nextInt(); // Avoid zero to allow xorShift rehash
- code = (h == 0) ? 1 : h;
- }
- }
-
- /**
- * The corresponding ThreadLocal class
- */
- static final class ThreadHashCode extends ThreadLocal<HashCode> {
- public HashCode initialValue() { return new HashCode(); }
- }
-
- /**
- * Static per-thread hash codes. Shared across all instances to
- * reduce ThreadLocal pollution and because adjustments due to
- * collisions in one table are likely to be appropriate for
- * others.
- */
- static final ThreadHashCode threadHashCode = new ThreadHashCode();
-
- /** Number of CPUS, to place bound on table size */
- static final int NCPU = Runtime.getRuntime().availableProcessors();
-
- /**
- * Table of cells. When non-null, size is a power of 2.
- */
- transient volatile Cell[] cells;
-
- /**
- * Base value, used mainly when there is no contention, but also as
- * a fallback during table initialization races. Updated via CAS.
- */
- transient volatile long base;
-
- /**
- * Spinlock (locked via CAS) used when resizing and/or creating Cells.
- */
- transient volatile int busy;
-
- /**
- * Package-private default constructor
- */
- Striped64_8() {
- }
-
- /**
- * CASes the base field.
- */
- final boolean casBase(long cmp, long val) {
- return UNSAFE.compareAndSwapLong(this, baseOffset, cmp, val);
- }
-
- /**
- * CASes the busy field from 0 to 1 to acquire lock.
- */
- final boolean casBusy() {
- return UNSAFE.compareAndSwapInt(this, busyOffset, 0, 1);
- }
-
- /**
- * Computes the function of current and new value. Subclasses
- * should open-code this update function for most uses, but the
- * virtualized form is needed within retryUpdate.
- *
- * @param currentValue the current value (of either base or a cell)
- * @param newValue the argument from a user update call
- * @return result of the update function
- */
- abstract long fn(long currentValue, long newValue);
-
- /**
- * Handles cases of updates involving initialization, resizing,
- * creating new Cells, and/or contention. See above for
- * explanation. This method suffers the usual non-modularity
- * problems of optimistic retry code, relying on rechecked sets of
- * reads.
- *
- * @param x the value
- * @param hc the hash code holder
- * @param wasUncontended false if CAS failed before call
- */
- final void retryUpdate(long x, HashCode hc, boolean wasUncontended) {
- int h = hc.code;
- boolean collide = false; // True if last slot nonempty
- for (;;) {
- Cell[] as; Cell a; int n; long v;
- if ((as = cells) != null && (n = as.length) > 0) {
- if ((a = as[(n - 1) & h]) == null) {
- if (busy == 0) { // Try to attach new Cell
- Cell r = new Cell(x); // Optimistically create
- if (busy == 0 && casBusy()) {
- boolean created = false;
- try { // Recheck under lock
- Cell[] rs; int m, j;
- if ((rs = cells) != null &&
- (m = rs.length) > 0 &&
- rs[j = (m - 1) & h] == null) {
- rs[j] = r;
- created = true;
- }
- } finally {
- busy = 0;
- }
- if (created)
- break;
- continue; // Slot is now non-empty
- }
- }
- collide = false;
- }
- else if (!wasUncontended) // CAS already known to fail
- wasUncontended = true; // Continue after rehash
- else if (a.cas(v = a.value, fn(v, x)))
- break;
- else if (n >= NCPU || cells != as)
- collide = false; // At max size or stale
- else if (!collide)
- collide = true;
- else if (busy == 0 && casBusy()) {
- try {
- if (cells == as) { // Expand table unless stale
- Cell[] rs = new Cell[n << 1];
- for (int i = 0; i < n; ++i)
- rs[i] = as[i];
- cells = rs;
- }
- } finally {
- busy = 0;
- }
- collide = false;
- continue; // Retry with expanded table
- }
- h ^= h << 13; // Rehash
- h ^= h >>> 17;
- h ^= h << 5;
- }
- else if (busy == 0 && cells == as && casBusy()) {
- boolean init = false;
- try { // Initialize table
- if (cells == as) {
- Cell[] rs = new Cell[2];
- rs[h & 1] = new Cell(x);
- cells = rs;
- init = true;
- }
- } finally {
- busy = 0;
- }
- if (init)
- break;
- }
- else if (casBase(v = base, fn(v, x)))
- break; // Fall back on using base
- }
- hc.code = h; // Record index for next time
- }
-
-
- /**
- * Sets base and all cells to the given value.
- */
- final void internalReset(long initialValue) {
- Cell[] as = cells;
- base = initialValue;
- if (as != null) {
- int n = as.length;
- for (int i = 0; i < n; ++i) {
- Cell a = as[i];
- if (a != null)
- a.value = initialValue;
- }
- }
- }
-
- // Unsafe mechanics
- private static final sun.misc.Unsafe UNSAFE;
- private static final long baseOffset;
- private static final long busyOffset;
- static {
- try {
- UNSAFE = getUnsafe();
- Class<?> sk = Striped64_8.class;
- baseOffset = UNSAFE.objectFieldOffset
- (sk.getDeclaredField("base"));
- busyOffset = UNSAFE.objectFieldOffset
- (sk.getDeclaredField("busy"));
- } catch (Exception e) {
- throw new Error(e);
- }
- }
-
- /**
- * Returns a sun.misc.Unsafe. Suitable for use in a 3rd party package.
- * Replace with a simple call to Unsafe.getUnsafe when integrating
- * into a jdk.
- *
- * @return a sun.misc.Unsafe
- */
- private static sun.misc.Unsafe getUnsafe() {
- try {
- return sun.misc.Unsafe.getUnsafe();
- } catch (SecurityException se) {
- try {
- return java.security.AccessController.doPrivileged
- (new java.security.PrivilegedExceptionAction<sun.misc.Unsafe>() {
- public sun.misc.Unsafe run() throws Exception {
- java.lang.reflect.Field f = sun.misc.Unsafe.class.getDeclaredField("theUnsafe");
- f.setAccessible(true);
-
- return (sun.misc.Unsafe)f.get(null);
- }
- });
- } catch (java.security.PrivilegedActionException e) {
- throw new RuntimeException("Could not initialize intrinsics",
- e.getCause());
- }
- }
- }
-
-}
\ No newline at end of file
+++ /dev/null
-/*
- * Written by Doug Lea with assistance from members of JCP JSR-166
- * Expert Group and released to the public domain, as explained at
- * http://creativecommons.org/publicdomain/zero/1.0/
- */
-
-/*
- * The latest version of the file corresponds to the following CVS commit:
- * http://gee.cs.oswego.edu/cgi-bin/viewcvs.cgi/jsr166/src/jdk7/java/util/concurrent/
- * ThreadLocalRandom.java.java?pathrev=1.3
- *
- * Note, that the repository above is JDK 7 based that is kept up-to-date too.
- * The main repository (JDK 8 based) uses JDK 8 features significantly that unavailable in JDK 7.
- */
-
-package org.jsr166;
-
-import java.util.*;
-
-/**
- * A random number generator isolated to the current thread. Like the
- * global {@link java.util.Random} generator used by the {@link
- * java.lang.Math} class, a {@code ThreadLocalRandom} is initialized
- * with an internally generated seed that may not otherwise be
- * modified. When applicable, use of {@code ThreadLocalRandom} rather
- * than shared {@code Random} objects in concurrent programs will
- * typically encounter much less overhead and contention. Use of
- * {@code ThreadLocalRandom} is particularly appropriate when multiple
- * tasks (for example, each a ForkJoinTask) use random numbers
- * in parallel in thread pools.
- *
- * <p>Usages of this class should typically be of the form:
- * {@code ThreadLocalRandom.current().nextX(...)} (where
- * {@code X} is {@code Int}, {@code Long}, etc).
- * When all usages are of this form, it is never possible to
- * accidently share a {@code ThreadLocalRandom} across multiple threads.
- *
- * <p>This class also provides additional commonly used bounded random
- * generation methods.
- *
- * @since 1.7
- * @author Doug Lea
- */
-@SuppressWarnings("ALL")
-public class ThreadLocalRandom8 extends Random {
- // same constants as Random, but must be redeclared because private
- private static final long multiplier = 0x5DEECE66DL;
- private static final long addend = 0xBL;
- private static final long mask = (1L << 48) - 1;
-
- /**
- * The random seed. We can't use super.seed.
- */
- private long rnd;
-
- /**
- * Initialization flag to permit calls to setSeed to succeed only
- * while executing the Random constructor. We can't allow others
- * since it would cause setting seed in one part of a program to
- * unintentionally impact other usages by the thread.
- */
- boolean initialized;
-
- // Padding to help avoid memory contention among seed updates in
- // different TLRs in the common case that they are located near
- // each other.
- private long pad0, pad1, pad2, pad3, pad4, pad5, pad6, pad7;
-
- /**
- * The actual ThreadLocal
- */
- private static final ThreadLocal<ThreadLocalRandom8> localRandom =
- new ThreadLocal<ThreadLocalRandom8>() {
- protected ThreadLocalRandom8 initialValue() {
- return new ThreadLocalRandom8();
- }
- };
-
-
- /**
- * Constructor called only by localRandom.initialValue.
- */
- ThreadLocalRandom8() {
- super();
- initialized = true;
- }
-
- /**
- * Returns the current thread's {@code ThreadLocalRandom}.
- *
- * @return the current thread's {@code ThreadLocalRandom}
- */
- public static ThreadLocalRandom8 current() {
- return localRandom.get();
- }
-
- /**
- * Throws {@code UnsupportedOperationException}. Setting seeds in
- * this generator is not supported.
- *
- * @throws UnsupportedOperationException always
- */
- public void setSeed(long seed) {
- if (initialized)
- throw new UnsupportedOperationException();
- rnd = (seed ^ multiplier) & mask;
- }
-
- protected int next(int bits) {
- rnd = (rnd * multiplier + addend) & mask;
- return (int) (rnd >>> (48-bits));
- }
-
- /**
- * Returns a pseudorandom, uniformly distributed value between the
- * given least value (inclusive) and bound (exclusive).
- *
- * @param least the least value returned
- * @param bound the upper bound (exclusive)
- * @return the next value
- * @throws IllegalArgumentException if least greater than or equal
- * to bound
- */
- public int nextInt(int least, int bound) {
- if (least >= bound)
- throw new IllegalArgumentException();
- return nextInt(bound - least) + least;
- }
-
- /**
- * Returns a pseudorandom, uniformly distributed value
- * between 0 (inclusive) and the specified value (exclusive).
- *
- * @param n the bound on the random number to be returned. Must be
- * positive.
- * @return the next value
- * @throws IllegalArgumentException if n is not positive
- */
- public long nextLong(long n) {
- if (n <= 0)
- throw new IllegalArgumentException("n must be positive");
- // Divide n by two until small enough for nextInt. On each
- // iteration (at most 31 of them but usually much less),
- // randomly choose both whether to include high bit in result
- // (offset) and whether to continue with the lower vs upper
- // half (which makes a difference only if odd).
- long offset = 0;
- while (n >= Integer.MAX_VALUE) {
- int bits = next(2);
- long half = n >>> 1;
- long nextn = ((bits & 2) == 0) ? half : n - half;
- if ((bits & 1) == 0)
- offset += n - nextn;
- n = nextn;
- }
- return offset + nextInt((int) n);
- }
-
- /**
- * Returns a pseudorandom, uniformly distributed value between the
- * given least value (inclusive) and bound (exclusive).
- *
- * @param least the least value returned
- * @param bound the upper bound (exclusive)
- * @return the next value
- * @throws IllegalArgumentException if least greater than or equal
- * to bound
- */
- public long nextLong(long least, long bound) {
- if (least >= bound)
- throw new IllegalArgumentException();
- return nextLong(bound - least) + least;
- }
-
- /**
- * Returns a pseudorandom, uniformly distributed {@code double} value
- * between 0 (inclusive) and the specified value (exclusive).
- *
- * @param n the bound on the random number to be returned. Must be
- * positive.
- * @return the next value
- * @throws IllegalArgumentException if n is not positive
- */
- public double nextDouble(double n) {
- if (!(n > 0))
- throw new IllegalArgumentException("n must be positive");
- return nextDouble() * n;
- }
-
- /**
- * Returns a pseudorandom, uniformly distributed value between the
- * given least value (inclusive) and bound (exclusive).
- *
- * @param least the least value returned
- * @param bound the upper bound (exclusive)
- * @return the next value
- * @throws IllegalArgumentException if least greater than or equal
- * to bound
- */
- public double nextDouble(double least, double bound) {
- if (least >= bound)
- throw new IllegalArgumentException();
- return nextDouble() * (bound - least) + least;
- }
-
- private static final long serialVersionUID = -5851777807851030925L;
-}
\ No newline at end of file
import java.util.Random;
import java.util.TreeMap;
import java.util.concurrent.Callable;
+import java.util.concurrent.atomic.LongAdder;
import javax.cache.configuration.Factory;
import org.apache.ignite.Ignite;
import org.apache.ignite.IgniteCache;
import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
import org.apache.ignite.transactions.Transaction;
-import org.jsr166.LongAdder8;
import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
import static org.apache.ignite.cache.CacheMode.PARTITIONED;
CacheStore store = cctx.store().configuredStore();
- long opened = ((LongAdder8)U.field(store, "opened")).sum();
- long closed = ((LongAdder8)U.field(store, "closed")).sum();
+ long opened = ((LongAdder)U.field(store, "opened")).sum();
+ long closed = ((LongAdder)U.field(store, "closed")).sum();
assert opened > 0;
assert closed > 0;
package org.apache.ignite.internal;
+import java.util.concurrent.ThreadLocalRandom;
import org.apache.ignite.IgniteException;
import org.apache.ignite.cluster.ClusterNode;
import org.apache.ignite.internal.managers.communication.GridIoMessage;
import org.apache.ignite.plugin.extensions.communication.Message;
import org.apache.ignite.spi.IgniteSpiException;
import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi;
-import org.jsr166.ThreadLocalRandom8;
/**
*
GridIoMessage ioMsg = (GridIoMessage)msg;
if (delayMessage(ioMsg.message(), ioMsg))
- U.sleep(ThreadLocalRandom8.current().nextInt(delayMillis()) + 1);
+ U.sleep(ThreadLocalRandom.current().nextInt(delayMillis()) + 1);
}
catch (IgniteInterruptedCheckedException e) {
throw new IgniteSpiException(e);
import java.util.concurrent.Callable;
import java.util.concurrent.ThreadLocalRandom;
import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.LongAdder;
import org.apache.ignite.IgniteDataStreamer;
+import org.apache.ignite.binary.BinaryObjectException;
+import org.apache.ignite.binary.BinaryReader;
+import org.apache.ignite.binary.BinaryTypeConfiguration;
+import org.apache.ignite.binary.BinaryWriter;
+import org.apache.ignite.binary.Binarylizable;
import org.apache.ignite.cache.CacheAtomicityMode;
import org.apache.ignite.cache.CacheMode;
import org.apache.ignite.cache.CacheWriteSynchronizationMode;
import org.apache.ignite.configuration.NearCacheConfiguration;
import org.apache.ignite.internal.IgniteInternalFuture;
import org.apache.ignite.internal.binary.BinaryMarshaller;
-import org.apache.ignite.binary.BinaryObjectException;
-import org.apache.ignite.binary.Binarylizable;
-import org.apache.ignite.binary.BinaryReader;
-import org.apache.ignite.binary.BinaryTypeConfiguration;
-import org.apache.ignite.binary.BinaryWriter;
import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
-import org.jsr166.LongAdder8;
import static org.apache.ignite.cache.CacheWriteSynchronizationMode.PRIMARY_SYNC;
public void testGetPut() throws Exception {
final AtomicBoolean flag = new AtomicBoolean();
- final LongAdder8 cnt = new LongAdder8();
+ final LongAdder cnt = new LongAdder();
try (IgniteDataStreamer<Object, Object> ldr = grid(0).dataStreamer(DEFAULT_CACHE_NAME)) {
IgniteInternalFuture<?> f = multithreadedAsync(
import java.util.concurrent.ThreadLocalRandom;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.LongAdder;
import org.apache.ignite.IgniteCache;
+import org.apache.ignite.binary.BinaryObject;
+import org.apache.ignite.binary.BinaryObjectException;
+import org.apache.ignite.binary.BinaryReader;
+import org.apache.ignite.binary.BinaryTypeConfiguration;
+import org.apache.ignite.binary.BinaryWriter;
+import org.apache.ignite.binary.Binarylizable;
import org.apache.ignite.cache.CacheAtomicityMode;
import org.apache.ignite.cache.CacheMode;
import org.apache.ignite.cache.CacheWriteSynchronizationMode;
import org.apache.ignite.configuration.IgniteConfiguration;
import org.apache.ignite.configuration.NearCacheConfiguration;
import org.apache.ignite.internal.IgniteInternalFuture;
-import org.apache.ignite.internal.processors.cache.IgniteCacheProxy;
import org.apache.ignite.internal.binary.BinaryMarshaller;
-import org.apache.ignite.binary.BinaryObjectException;
-import org.apache.ignite.binary.Binarylizable;
-import org.apache.ignite.binary.BinaryObject;
-import org.apache.ignite.binary.BinaryReader;
-import org.apache.ignite.binary.BinaryTypeConfiguration;
-import org.apache.ignite.binary.BinaryWriter;
+import org.apache.ignite.internal.processors.cache.IgniteCacheProxy;
import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
-import org.jsr166.LongAdder8;
import static org.apache.ignite.cache.CacheWriteSynchronizationMode.PRIMARY_SYNC;
@SuppressWarnings("BusyWait") public void testGetPut() throws Exception {
final AtomicBoolean flag = new AtomicBoolean();
- final LongAdder8 cnt = new LongAdder8();
+ final LongAdder cnt = new LongAdder();
IgniteInternalFuture<?> f = multithreadedAsync(
new Callable<Object>() {
import java.util.List;
import java.util.concurrent.Callable;
import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ThreadLocalRandom;
import java.util.concurrent.atomic.AtomicInteger;
import org.apache.ignite.Ignite;
import org.apache.ignite.IgniteCache;
import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
import org.apache.ignite.testframework.GridTestUtils;
import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
-import org.jsr166.ThreadLocalRandom8;
import static org.apache.ignite.cache.CacheAtomicityMode.ATOMIC;
import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
log.info("Started [node=" + ignite.name() + ", left=" + latch.getCount() + ']');
- ThreadLocalRandom8 rnd = ThreadLocalRandom8.current();
+ ThreadLocalRandom rnd = ThreadLocalRandom.current();
while (latch.getCount() > 0) {
Thread.sleep(1000);
import java.util.HashMap;
import java.util.Map;
+import java.util.concurrent.ThreadLocalRandom;
import org.apache.ignite.IgniteCache;
import org.apache.ignite.cluster.ClusterNode;
import org.apache.ignite.configuration.CacheConfiguration;
import org.apache.ignite.transactions.Transaction;
import org.apache.ignite.transactions.TransactionConcurrency;
import org.apache.ignite.transactions.TransactionIsolation;
-import org.jsr166.ThreadLocalRandom8;
import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC;
Map<Integer, String> secondCheck = new HashMap<>();
for (int i = 0; i < TX_CNT; i++) {
- int grid = ThreadLocalRandom8.current().nextInt(nodeCount());
+ int grid = ThreadLocalRandom.current().nextInt(nodeCount());
IgniteCache<Integer, String> first = grid(grid).cache(FIRST_CACHE);
IgniteCache<Integer, String> second = grid(grid).cache(SECOND_CACHE);
try (Transaction tx = grid(grid).transactions().txStart(concurrency, isolation)) {
try {
- int size = ThreadLocalRandom8.current().nextInt(24) + 1;
+ int size = ThreadLocalRandom.current().nextInt(24) + 1;
for (int k = 0; k < size; k++) {
- boolean rnd = ThreadLocalRandom8.current().nextBoolean();
+ boolean rnd = ThreadLocalRandom.current().nextBoolean();
IgniteCache<Integer, String> cache = rnd ? first : second;
Map<Integer, String> check = rnd ? firstCheck : secondCheck;
package org.apache.ignite.internal.processors.cache.eviction.sorted;
import java.util.concurrent.Callable;
+import java.util.concurrent.ThreadLocalRandom;
import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.LongAdder;
import org.apache.ignite.Ignite;
import org.apache.ignite.IgniteCache;
import org.apache.ignite.cache.CacheAtomicityMode;
import org.apache.ignite.configuration.IgniteConfiguration;
import org.apache.ignite.internal.util.typedef.internal.U;
import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
-import org.jsr166.LongAdder8;
-import org.jsr166.ThreadLocalRandom8;
/**
* {@link SortedEvictionPolicy} performance test.
private static final int P_GET = 30;
/** Rnd. */
- private static final ThreadLocalRandom8 RND = ThreadLocalRandom8.current();
+ private static final ThreadLocalRandom RND = ThreadLocalRandom.current();
/** Ignite. */
private static Ignite ignite;
* Tests throughput.
*/
public void testThroughput() throws Exception {
- final LongAdder8 cnt = new LongAdder8();
+ final LongAdder cnt = new LongAdder();
final AtomicBoolean finished = new AtomicBoolean();
final int pPut = P_PUT;
import java.util.concurrent.ThreadLocalRandom;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.LongAdder;
import org.apache.ignite.Ignite;
import org.apache.ignite.IgniteCache;
import org.apache.ignite.cache.CacheAtomicityMode;
import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
import org.apache.ignite.transactions.Transaction;
import org.apache.ignite.transactions.TransactionIsolation;
-import org.jsr166.LongAdder8;
import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.DFLT_STORE_DIR;
import static org.apache.ignite.transactions.TransactionConcurrency.PESSIMISTIC;
final CyclicBarrier cyclicBarrier = new CyclicBarrier(THREADS_CNT);
final AtomicBoolean interrupt = new AtomicBoolean(false);
- final LongAdder8 operationCnt = new LongAdder8();
+ final LongAdder operationCnt = new LongAdder();
final IgniteCache<Long, TestEntity> cache = g.cache(CACHE_NAME);
package org.apache.ignite.internal.processors.cache.persistence.db.file;
+import java.util.concurrent.ThreadLocalRandom;
import java.util.concurrent.atomic.AtomicReference;
import org.apache.ignite.Ignite;
import org.apache.ignite.IgniteCache;
import org.apache.ignite.internal.processors.cache.persistence.file.AsyncFileIOFactory;
import org.apache.ignite.internal.util.typedef.internal.U;
import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
-import org.jsr166.ThreadLocalRandom8;
/**
* Test what interruptions of writing threads do not affect PDS.
Runnable clo = new Runnable() {
@Override
public void run() {
- cache.get(ThreadLocalRandom8.current().nextInt(maxKey / 5));
+ cache.get(ThreadLocalRandom.current().nextInt(maxKey / 5));
}
};
for (int i = 0; i < workers.length; i++) {
IgniteCache<Object, Object> cache = ignite.cache(CACHE_NAME);
while (!stop)
- cache.put(ThreadLocalRandom8.current().nextInt(maxKey), payload);
+ cache.put(ThreadLocalRandom.current().nextInt(maxKey), payload);
}
};
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.ThreadLocalRandom;
import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.LongAdder;
import javax.cache.CacheException;
import org.apache.ignite.Ignite;
import org.apache.ignite.IgniteCache;
import org.apache.ignite.transactions.TransactionIsolation;
import org.apache.ignite.transactions.TransactionOptimisticException;
import org.apache.ignite.transactions.TransactionTimeoutException;
-import org.jsr166.LongAdder8;
import static java.lang.Thread.sleep;
import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
final TransactionConcurrency[] TC_VALS = TransactionConcurrency.values();
final TransactionIsolation[] TI_VALS = TransactionIsolation.values();
- final LongAdder8 cntr0 = new LongAdder8();
- final LongAdder8 cntr1 = new LongAdder8();
- final LongAdder8 cntr2 = new LongAdder8();
- final LongAdder8 cntr3 = new LongAdder8();
+ final LongAdder cntr0 = new LongAdder();
+ final LongAdder cntr1 = new LongAdder();
+ final LongAdder cntr2 = new LongAdder();
+ final LongAdder cntr3 = new LongAdder();
final IgniteInternalFuture<?> fut = multithreadedAsync(new Runnable() {
@Override public void run() {
package org.apache.ignite.internal.processors.datastreamer;
import java.util.concurrent.Callable;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.atomic.LongAdder;
import org.apache.ignite.Ignite;
import org.apache.ignite.IgniteDataStreamer;
import org.apache.ignite.configuration.CacheConfiguration;
import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
-import org.jsr166.LongAdder8;
-import org.jsr166.ThreadLocalRandom8;
import static org.apache.ignite.cache.CacheMode.PARTITIONED;
import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC;
super.beforeTestsStarted();
for (int i = 0; i < vals.length; i++) {
- int valLen = ThreadLocalRandom8.current().nextInt(128, 512);
+ int valLen = ThreadLocalRandom.current().nextInt(128, 512);
StringBuilder sb = new StringBuilder();
for (int j = 0; j < valLen; j++)
- sb.append('a' + ThreadLocalRandom8.current().nextInt(20));
+ sb.append('a' + ThreadLocalRandom.current().nextInt(20));
vals[i] = sb.toString();
ldr.receiver(DataStreamerCacheUpdaters.<Integer, String>batchedSorted());
ldr.autoFlushFrequency(0);
- final LongAdder8 cnt = new LongAdder8();
+ final LongAdder cnt = new LongAdder();
long start = U.currentTimeMillis();
multithreaded(new Callable<Object>() {
@SuppressWarnings("InfiniteLoopStatement")
@Override public Object call() throws Exception {
- ThreadLocalRandom8 rnd = ThreadLocalRandom8.current();
+ ThreadLocalRandom rnd = ThreadLocalRandom.current();
while (true) {
int i = rnd.nextInt(ENTRY_CNT);
package org.apache.ignite.internal.processors.igfs;
import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.UUID;
import java.util.concurrent.Callable;
+import java.util.concurrent.ThreadLocalRandom;
import org.apache.ignite.Ignite;
import org.apache.ignite.cache.CacheMode;
import org.apache.ignite.cache.CacheWriteSynchronizationMode;
import org.apache.ignite.cluster.ClusterNode;
import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.DataRegionConfiguration;
import org.apache.ignite.configuration.DataStorageConfiguration;
import org.apache.ignite.configuration.FileSystemConfiguration;
import org.apache.ignite.configuration.IgniteConfiguration;
-import org.apache.ignite.configuration.DataRegionConfiguration;
import org.apache.ignite.configuration.NearCacheConfiguration;
import org.apache.ignite.igfs.IgfsGroupDataBlocksKeyMapper;
import org.apache.ignite.igfs.IgfsInputStream;
import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
import org.apache.ignite.testframework.GridTestUtils;
-import org.jsr166.ThreadLocalRandom8;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Map;
-import java.util.UUID;
import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
import static org.apache.ignite.cache.CacheMode.PARTITIONED;
private Collection<IgfsFile> write() throws Exception {
Collection<IgfsFile> res = new HashSet<>(FILES_CNT, 1.0f);
- ThreadLocalRandom8 rand = ThreadLocalRandom8.current();
+ ThreadLocalRandom rand = ThreadLocalRandom.current();
for (int i = 0; i < FILES_CNT; i++) {
// Create empty file locally.
import java.util.concurrent.Callable;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
+import java.util.concurrent.atomic.LongAdder;
import org.apache.ignite.IgniteCheckedException;
import org.apache.ignite.internal.IgniteInternalFuture;
import org.apache.ignite.lang.IgniteInClosure;
import org.jsr166.ConcurrentLinkedDeque8;
-import org.jsr166.LongAdder8;
/**
*
* @throws InterruptedException If failed.
*/
public static void main(String[] args) throws InterruptedException {
- final LongAdder8 cnt = new LongAdder8();
+ final LongAdder cnt = new LongAdder();
final ConcurrentLinkedDeque8<GridFutureAdapter<Object>> futs = new ConcurrentLinkedDeque8<>();
import java.util.Collection;
import java.util.UUID;
import java.util.concurrent.Callable;
+import java.util.concurrent.ThreadLocalRandom;
import java.util.concurrent.atomic.AtomicReference;
import org.apache.ignite.IgniteCheckedException;
import org.apache.ignite.IgniteSystemProperties;
import org.apache.ignite.internal.util.lang.GridAbsPredicate;
import org.apache.ignite.testframework.GridTestUtils;
import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
-import org.jsr166.ThreadLocalRandom8;
/**
*
for (; ; ) {
int len = Math.min(DATA.length - bytesWritten,
- ThreadLocalRandom8.current().nextInt(256) + 1);
+ ThreadLocalRandom.current().nextInt(256) + 1);
space.write(DATA, bytesWritten, len, 0);
for (; ; ) {
int len = Math.min(DATA.length - bytesRead,
- ThreadLocalRandom8.current().nextInt(32) + 1);
+ ThreadLocalRandom.current().nextInt(32) + 1);
int len0 = space.read(buf, bytesRead, len, 0);
import java.util.concurrent.Callable;
import java.util.concurrent.LinkedBlockingDeque;
import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.atomic.LongAdder;
import org.apache.ignite.internal.util.GridTimer;
import org.apache.ignite.internal.util.typedef.X;
import org.apache.ignite.testframework.GridTestUtils;
-import org.jsr166.LongAdder8;
/**
* Blocking queue performance benchmark.
X.println(">>> Starting test for: " + testName);
- final LongAdder8 adder = new LongAdder8();
+ final LongAdder adder = new LongAdder();
GridTestUtils.runMultiThreaded(new Callable<Object>() {
@Override public Object call() throws Exception {
import java.util.concurrent.Callable;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ThreadLocalRandom;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
import org.apache.ignite.internal.IgniteInternalFuture;
import org.apache.ignite.internal.util.typedef.X;
import org.apache.ignite.testframework.GridTestUtils;
import org.jsr166.ConcurrentHashMap8;
-import org.jsr166.ThreadLocalRandom8;
/**
*
GridTestUtils.runMultiThreaded(new Callable<Object>() {
@Override public Object call() throws Exception {
- ThreadLocalRandom8 rnd = ThreadLocalRandom8.current();
+ ThreadLocalRandom rnd = ThreadLocalRandom.current();
for (int i = 0; i < iterCnt; i++) {
// Put random.
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.LinkedBlockingDeque;
import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.ThreadLocalRandom;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
import org.apache.ignite.testframework.GridTestUtils;
import org.jetbrains.annotations.Nullable;
import org.jsr166.ConcurrentLinkedDeque8;
-import org.jsr166.ThreadLocalRandom8;
/**
* Tests synchronization performance vs. lock.
int lim = 10000;
for (int i = 0; i < arr.length; i++)
- arr[i] = ThreadLocalRandom8.current().nextInt(lim);
+ arr[i] = ThreadLocalRandom.current().nextInt(lim);
Arrays.sort(arr);
for (int i = 0; i < MAX; i++) {
if (sort)
- Arrays.binarySearch(arr, ThreadLocalRandom8.current().nextInt(lim));
+ Arrays.binarySearch(arr, ThreadLocalRandom.current().nextInt(lim));
else
- F.contains(arr, ThreadLocalRandom8.current().nextInt(lim));
+ F.contains(arr, ThreadLocalRandom.current().nextInt(lim));
}
long time = System.currentTimeMillis() - start;
int lim = 10000;
for (int i = 0; i < arr.length; i++)
- arr[i] = ThreadLocalRandom8.current().nextLong(lim);
+ arr[i] = ThreadLocalRandom.current().nextLong(lim);
Arrays.sort(arr);
for (int i = 0; i < MAX; i++) {
if (sort)
- Arrays.binarySearch(arr, ThreadLocalRandom8.current().nextInt(lim));
+ Arrays.binarySearch(arr, ThreadLocalRandom.current().nextInt(lim));
else
- F.contains(arr, ThreadLocalRandom8.current().nextInt(lim));
+ F.contains(arr, ThreadLocalRandom.current().nextInt(lim));
}
long time = System.currentTimeMillis() - start;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.LongAdder;
import org.apache.ignite.IgniteCheckedException;
import org.apache.ignite.internal.IgniteInternalFuture;
import org.apache.ignite.internal.util.future.GridFutureAdapter;
import org.jsr166.ConcurrentLinkedDeque8;
-import org.jsr166.LongAdder8;
/**
*
* @throws InterruptedException If failed.
*/
public static void main(String[] args) throws InterruptedException {
- final LongAdder8 cnt = new LongAdder8();
+ final LongAdder cnt = new LongAdder();
final ConcurrentLinkedDeque8<GridFutureAdapter<Object>> futs = new ConcurrentLinkedDeque8<>();
import java.util.concurrent.Callable;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.LongAdder;
import org.apache.ignite.internal.util.GridCircularBuffer;
import org.apache.ignite.internal.util.typedef.internal.U;
import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
import org.jsr166.ConcurrentLinkedDeque8;
-import org.jsr166.LongAdder8;
/**
*
int size = 256 * 1024;
final GridCircularBuffer<Integer> buf = new GridCircularBuffer<>(size);
- final LongAdder8 cnt = new LongAdder8();
+ final LongAdder cnt = new LongAdder();
final AtomicBoolean finished = new AtomicBoolean();
multithreadedAsync(new Callable<Object>() {
public void testDequeueThroughput() throws Exception {
final ConcurrentLinkedDeque8<Integer> buf = new ConcurrentLinkedDeque8<>();
- final LongAdder8 cnt = new LongAdder8();
+ final LongAdder cnt = new LongAdder();
final AtomicBoolean finished = new AtomicBoolean();
multithreadedAsync(new Callable<Object>() {
final int size = 256 * 1024;
final ArrayBlockingQueue<Integer> buf = new ArrayBlockingQueue<>(size);
- final LongAdder8 cnt = new LongAdder8();
+ final LongAdder cnt = new LongAdder();
final AtomicBoolean finished = new AtomicBoolean();
multithreadedAsync(new Callable<Object>() {
final int size = 256 * 1024;
final ArrayBlockingQueue<Integer> buf = new ArrayBlockingQueue<>(size);
- final LongAdder8 cnt = new LongAdder8();
+ final LongAdder cnt = new LongAdder();
final AtomicBoolean finished = new AtomicBoolean();
multithreadedAsync(new Callable<Object>() {
import java.util.Map;
import java.util.concurrent.Callable;
import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ThreadLocalRandom;
import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
import org.jsr166.ConcurrentHashMap8;
-import org.jsr166.ThreadLocalRandom8;
/**
* Tests for {@link org.jsr166.ConcurrentHashMap8}.
multithreaded(new Callable<Object>() {
@Override public Object call() throws Exception {
- ThreadLocalRandom8 rnd = ThreadLocalRandom8.current();
+ ThreadLocalRandom rnd = ThreadLocalRandom.current();
for (int i = 0; i < iterCnt; i++) {
// Put random.
import java.util.concurrent.ExecutorCompletionService;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
+import java.util.concurrent.atomic.LongAdder;
import javax.cache.integration.CacheLoaderException;
import org.apache.ignite.Ignite;
import org.apache.ignite.IgniteCache;
import org.apache.ignite.lang.IgniteBiInClosure;
import org.apache.ignite.resources.IgniteInstanceResource;
import org.apache.ignite.resources.LoggerResource;
-import org.jsr166.LongAdder8;
/**
* Accenture cache store.
assert cache != null;
- final LongAdder8 adder = new LongAdder8();
+ final LongAdder adder = new LongAdder();
for (int i = 0; i < numThreads; i++) {
final int threadId = i;
import java.util.UUID;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.Semaphore;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.atomic.LongAdder;
import org.apache.ignite.IgniteCheckedException;
import org.apache.ignite.cluster.ClusterNode;
import org.apache.ignite.internal.IgniteInterruptedCheckedException;
import org.apache.ignite.testframework.GridLoadTestUtils;
import org.jetbrains.annotations.Nullable;
import org.jsr166.ConcurrentHashMap8;
-import org.jsr166.LongAdder8;
-import org.jsr166.ThreadLocalRandom8;
import static java.util.concurrent.TimeUnit.MILLISECONDS;
import static org.apache.ignite.internal.managers.communication.GridIoPolicy.PUBLIC_POOL;
public static final int TEST_TOPIC = 1;
/** */
- private static final LongAdder8 msgCntr = new LongAdder8();
+ private static final LongAdder msgCntr = new LongAdder();
/** */
private static final Map<IgniteUuid, CountDownLatch> latches = new ConcurrentHashMap8<>();
*
*/
static {
- ThreadLocalRandom8 rnd = ThreadLocalRandom8.current();
+ ThreadLocalRandom rnd = ThreadLocalRandom.current();
arrs = new byte[64][];
GridIoManager io = g.context().io();
- Random rnd = ThreadLocalRandom8.current();
+ Random rnd = ThreadLocalRandom.current();
IgniteUuid msgId = IgniteUuid.randomUuid();
import java.util.concurrent.Semaphore;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicReference;
+import java.util.concurrent.atomic.LongAdder;
import org.apache.ignite.IgniteCheckedException;
import org.apache.ignite.cluster.ClusterNode;
import org.apache.ignite.configuration.IgniteConfiguration;
import org.apache.ignite.testframework.GridTestUtils;
import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
import org.jsr166.ConcurrentHashMap8;
-import org.jsr166.LongAdder8;
import static org.apache.ignite.internal.managers.communication.GridIoPolicy.PUBLIC_POOL;
info("Messages: " + CONCUR_MSGS);
final Semaphore sem = new Semaphore(CONCUR_MSGS);
- final LongAdder8 msgCntr = new LongAdder8();
+ final LongAdder msgCntr = new LongAdder();
final String topic = "test-topic";
final GridIoManager snd = sndKernal.context().io();
final GridIoManager rcv = rcvKernal.context().io();
- final LongAdder8 msgCntr = new LongAdder8();
+ final LongAdder msgCntr = new LongAdder();
final Integer topic = 1;
info("Messages: " + CONCUR_MSGS);
final Semaphore sem = new Semaphore(CONCUR_MSGS);
- final LongAdder8 msgCntr = new LongAdder8();
+ final LongAdder msgCntr = new LongAdder();
final String topic = "test-topic";
import java.util.UUID;
import java.util.concurrent.Callable;
+import java.util.concurrent.ThreadLocalRandom;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicLong;
import javax.cache.event.CacheEntryEvent;
import org.apache.ignite.internal.util.typedef.X;
import org.apache.ignite.internal.util.typedef.internal.U;
import org.jetbrains.annotations.Nullable;
-import org.jsr166.ThreadLocalRandom8;
import static org.apache.ignite.events.EventType.EVT_CACHE_OBJECT_PUT;
import static org.apache.ignite.loadtests.util.GridLoadTestArgs.CACHE_NAME;
IgniteInternalFuture<Long> genFut = runMultiThreadedAsync(new Callable<Object>() {
@Override public Object call() throws Exception {
byte[] val = new byte[valSize];
- ThreadLocalRandom8 rnd = ThreadLocalRandom8.current();
+ ThreadLocalRandom rnd = ThreadLocalRandom.current();
while (!stop.get() && !Thread.currentThread().isInterrupted()) {
Integer key = rnd.nextInt(keyRange);
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.LongAdder;
import org.apache.ignite.internal.util.GridBoundedConcurrentLinkedHashSet;
import org.apache.ignite.internal.util.typedef.X;
import org.apache.ignite.lang.IgniteUuid;
import org.apache.ignite.loadtests.util.GridCumulativeAverage;
-import org.jsr166.LongAdder8;
import static org.jsr166.ConcurrentLinkedHashMap.QueuePolicy;
import static org.jsr166.ConcurrentLinkedHashMap.QueuePolicy.SINGLE_Q;
X.println("Set: " + set);
- final LongAdder8 execCnt = new LongAdder8();
+ final LongAdder execCnt = new LongAdder();
final AtomicBoolean finish = new AtomicBoolean();
import java.util.concurrent.Callable;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
+import java.util.concurrent.atomic.LongAdder;
import org.apache.ignite.Ignite;
import org.apache.ignite.IgniteCompute;
import org.apache.ignite.IgniteException;
import org.apache.ignite.testframework.GridFileLock;
import org.apache.ignite.testframework.GridLoadTestUtils;
import org.jetbrains.annotations.Nullable;
-import org.jsr166.LongAdder8;
/**
*
private static Ignite g;
/** Transaction count. */
- private static LongAdder8 txCnt = new LongAdder8();
+ private static LongAdder txCnt = new LongAdder();
/** Finish flag. */
private static volatile boolean finish;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.Semaphore;
+import java.util.concurrent.atomic.LongAdder;
import org.apache.ignite.Ignite;
import org.apache.ignite.IgniteCompute;
import org.apache.ignite.IgniteException;
import org.apache.ignite.testframework.GridFileLock;
import org.apache.ignite.testframework.GridLoadTestUtils;
import org.jetbrains.annotations.Nullable;
-import org.jsr166.LongAdder8;
/**
*
private static Ignite g;
/** Transaction count. */
- private static LongAdder8 txCnt = new LongAdder8();
+ private static LongAdder txCnt = new LongAdder();
/** Finish flag. */
private static volatile boolean finish;
import java.util.Map;
import java.util.concurrent.Callable;
import java.util.concurrent.Semaphore;
+import java.util.concurrent.atomic.LongAdder;
import org.apache.ignite.Ignite;
import org.apache.ignite.cluster.ClusterNode;
import org.apache.ignite.compute.ComputeJob;
import org.apache.ignite.testframework.GridLoadTestUtils;
import org.apache.ignite.testframework.GridTestUtils;
import org.jetbrains.annotations.Nullable;
-import org.jsr166.LongAdder8;
import static org.apache.ignite.compute.ComputeJobResultPolicy.REDUCE;
final int duration = args.length > 2 ? Integer.parseInt(args[2]) : 0;
final String outputFileName = args.length > 3 ? args[3] : null;
- final LongAdder8 execCnt = new LongAdder8();
+ final LongAdder execCnt = new LongAdder();
try {
final Ignite g = G.start("modules/tests/config/grid-job-load.xml");
* @param iterCntr Iteration counter.
*/
private static void runTest(final Ignite g, int threadCnt, int taskCnt, long dur,
- final LongAdder8 iterCntr) {
+ final LongAdder iterCntr) {
final Semaphore sem = new Semaphore(taskCnt);
final IgniteInClosure<IgniteFuture> lsnr = new CI1<IgniteFuture>() {
import java.util.UUID;
import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ThreadLocalRandom;
import java.util.concurrent.atomic.AtomicReference;
import org.apache.ignite.Ignite;
import org.apache.ignite.IgniteMessaging;
import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
-import org.jsr166.ThreadLocalRandom8;
import org.junit.Assert;
/**
final List<String> msgs = Lists.newArrayList();
for (int i = 0; i < 1000; i++)
- msgs.add(String.valueOf(ThreadLocalRandom8.current().nextInt()));
+ msgs.add(String.valueOf(ThreadLocalRandom.current().nextInt()));
return msgs;
}
import java.util.concurrent.Callable;
import java.util.concurrent.ConcurrentLinkedQueue;
import java.util.concurrent.CyclicBarrier;
+import java.util.concurrent.ThreadLocalRandom;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
import org.apache.hadoop.conf.Configuration;
import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
import org.apache.ignite.testframework.GridTestUtils;
import org.jetbrains.annotations.Nullable;
-import org.jsr166.ThreadLocalRandom8;
import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
import static org.apache.ignite.cache.CacheMode.PARTITIONED;
FSDataInputStream is = null;
try {
- int pos = ThreadLocalRandom8.current().nextInt(2048);
+ int pos = ThreadLocalRandom.current().nextInt(2048);
try {
is = fs.open(file);
import java.util.concurrent.Callable;
import java.util.concurrent.ConcurrentLinkedQueue;
import java.util.concurrent.CyclicBarrier;
+import java.util.concurrent.ThreadLocalRandom;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicReference;
import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
import org.apache.ignite.testframework.GridTestUtils;
import org.jetbrains.annotations.Nullable;
-import org.jsr166.ThreadLocalRandom8;
import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
import static org.apache.ignite.cache.CacheMode.PARTITIONED;
FSDataInputStream is = null;
try {
- int pos = ThreadLocalRandom8.current().nextInt(2048);
+ int pos = ThreadLocalRandom.current().nextInt(2048);
try {
is = fs.open(file);
import java.util.concurrent.Callable;
import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.LongAdder;
import org.apache.ignite.internal.IgniteInternalFuture;
import org.apache.ignite.internal.processors.hadoop.taskexecutor.HadoopExecutorService;
import org.apache.ignite.internal.util.typedef.X;
import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
-import org.jsr166.LongAdder8;
/**
*
final int loops = 5000;
int threads = 17;
- final LongAdder8 sum = new LongAdder8();
+ final LongAdder sum = new LongAdder();
multithreaded(new Callable<Object>() {
@Override public Object call() throws Exception {
for (int i = 0; i < 5; i++) {
final HadoopExecutorService exec = new HadoopExecutorService(log, "_GRID_NAME_", 10, 5);
- final LongAdder8 sum = new LongAdder8();
+ final LongAdder sum = new LongAdder();
final AtomicBoolean finish = new AtomicBoolean();
import java.util.List;
import java.util.Map;
import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.atomic.LongAdder;
import java.util.concurrent.locks.Lock;
import java.util.concurrent.locks.ReadWriteLock;
import java.util.concurrent.locks.ReentrantReadWriteLock;
import org.h2.value.DataType;
import org.jetbrains.annotations.Nullable;
import org.jsr166.ConcurrentHashMap8;
-import org.jsr166.LongAdder8;
import static org.apache.ignite.cache.CacheMode.PARTITIONED;
import static org.apache.ignite.internal.processors.query.h2.opt.GridH2KeyValueRowOnheap.DEFAULT_COLUMNS_COUNT;
private IndexColumn affKeyCol;
/** */
- private final LongAdder8 size = new LongAdder8();
+ private final LongAdder size = new LongAdder();
/** */
private final H2RowFactory rowFactory;
package org.apache.ignite.internal.processors.query.h2.twostep;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.LinkedBlockingDeque;
+import java.util.concurrent.atomic.LongAdder;
import org.apache.ignite.IgniteException;
import org.apache.ignite.IgniteLogger;
import org.apache.ignite.internal.IgniteInterruptedCheckedException;
import org.apache.ignite.internal.util.typedef.internal.U;
import org.apache.ignite.internal.util.worker.GridWorker;
import org.jetbrains.annotations.Nullable;
-import org.jsr166.LongAdder8;
-
-import java.util.concurrent.BlockingQueue;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.LinkedBlockingDeque;
/**
* Worker for lazy query execution.
private static final ThreadLocal<MapQueryLazyWorker> LAZY_WORKER = new ThreadLocal<>();
/** Active lazy worker count (for testing purposes). */
- private static final LongAdder8 ACTIVE_CNT = new LongAdder8();
+ private static final LongAdder ACTIVE_CNT = new LongAdder();
/** Task to be executed. */
private final BlockingQueue<Runnable> tasks = new LinkedBlockingDeque<>();
import java.util.Random;
import java.util.concurrent.Callable;
import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.LongAdder;
import org.apache.ignite.Ignite;
import org.apache.ignite.IgniteCache;
import org.apache.ignite.cache.CacheAtomicityMode;
import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
-import org.jsr166.LongAdder8;
/**
*
final AtomicBoolean end = new AtomicBoolean();
- final LongAdder8 puts = new LongAdder8();
+ final LongAdder puts = new LongAdder();
IgniteInternalFuture<?> fut0 = multithreadedAsync(new Callable<Void>() {
@Override public Void call() throws Exception {
}
}, 10);
- final LongAdder8 qrys = new LongAdder8();
+ final LongAdder qrys = new LongAdder();
IgniteInternalFuture<?> fut1 = multithreadedAsync(new Callable<Void>() {
@Override public Void call() throws Exception {
import java.util.NavigableMap;
import java.util.TreeMap;
import java.util.UUID;
+import java.util.concurrent.ThreadLocalRandom;
import javax.cache.Cache;
import javax.cache.CacheException;
import org.apache.ignite.Ignite;
import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
import org.apache.ignite.util.AttributeNodeFilter;
-import org.jsr166.ThreadLocalRandom8;
import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
import static org.apache.ignite.cache.CacheRebalanceMode.SYNC;
DepositKey dk = new DepositKey(depositId++, new ClientKey(clientId, regionId));
Deposit depo = new Deposit();
- depo.amount = ThreadLocalRandom8.current().nextLong(1_000_001);
+ depo.amount = ThreadLocalRandom.current().nextLong(1_000_001);
depStr.addData(dk, depo);
}
import java.util.Map;
import java.util.concurrent.Callable;
import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ThreadLocalRandom;
import javax.cache.CacheException;
import org.apache.ignite.Ignite;
import org.apache.ignite.IgniteCache;
import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
import org.apache.ignite.testframework.GridTestUtils;
import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
-import org.jsr166.ThreadLocalRandom8;
import static java.util.concurrent.TimeUnit.MILLISECONDS;
import static org.apache.ignite.events.EventType.EVT_CACHE_QUERY_EXECUTED;
for (String s1 : a) {
for (String s2 : b) {
if (!s1.equals(s2)) {
- String end = ends[ThreadLocalRandom8.current().nextInt(ends.length)];
+ String end = ends[ThreadLocalRandom.current().nextInt(ends.length)];
res.add(s1 + " " + s2 + end);
}
import java.util.List;
import java.util.Map;
import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.atomic.LongAdder;
import javax.cache.Cache;
import org.apache.ignite.IgniteCache;
import org.apache.ignite.IgniteDataStreamer;
-import org.jsr166.LongAdder8;
import static org.yardstickframework.BenchmarkUtils.println;
final List<Thread> threads = new ArrayList<>(THREADS);
- final LongAdder8 addedCnt = new LongAdder8();
+ final LongAdder addedCnt = new LongAdder();
int delta = (int)((KEYS_HI + Math.abs(KEYS_LO)) / THREADS);