diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/searchablesnapshots/SearchableSnapshotShardStats.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/searchablesnapshots/SearchableSnapshotShardStats.java index 4b510e256d2e0..319559516045c 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/searchablesnapshots/SearchableSnapshotShardStats.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/searchablesnapshots/SearchableSnapshotShardStats.java @@ -11,6 +11,7 @@ import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.io.stream.Writeable; +import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.xcontent.ToXContentObject; import org.elasticsearch.common.xcontent.XContentBuilder; @@ -125,7 +126,9 @@ public static class CacheIndexInputStats implements Writeable, ToXContentObject private final String fileExt; private final long numFiles; - private final long totalSize; + private final ByteSizeValue totalSize; + private final ByteSizeValue minSize; + private final ByteSizeValue maxSize; private final long openCount; private final long closeCount; @@ -144,7 +147,8 @@ public static class CacheIndexInputStats implements Writeable, ToXContentObject private final Counter blobStoreBytesRequested; private final long currentIndexCacheFills; - public CacheIndexInputStats(String fileExt, long numFiles, long totalSize, long openCount, long closeCount, + public CacheIndexInputStats(String fileExt, long numFiles, ByteSizeValue totalSize, ByteSizeValue minSize, ByteSizeValue maxSize, + long openCount, long closeCount, Counter forwardSmallSeeks, Counter backwardSmallSeeks, Counter forwardLargeSeeks, Counter backwardLargeSeeks, Counter contiguousReads, Counter nonContiguousReads, @@ -154,6 +158,8 @@ public CacheIndexInputStats(String fileExt, long numFiles, long totalSize, long this.fileExt = fileExt; this.numFiles = numFiles; this.totalSize = totalSize; + this.minSize = minSize; + this.maxSize = maxSize; this.openCount = openCount; this.closeCount = closeCount; this.forwardSmallSeeks = forwardSmallSeeks; @@ -180,7 +186,15 @@ public CacheIndexInputStats(String fileExt, long numFiles, long totalSize, long } this.fileExt = in.readString(); this.numFiles = in.readVLong(); - this.totalSize = in.readVLong(); + if (in.getVersion().before(Version.V_7_13_0)) { + this.totalSize = new ByteSizeValue(in.readVLong()); + this.minSize = ByteSizeValue.ZERO; + this.maxSize = ByteSizeValue.ZERO; + } else { + this.totalSize = new ByteSizeValue(in); + this.minSize = new ByteSizeValue(in); + this.maxSize = new ByteSizeValue(in); + } this.openCount = in.readVLong(); this.closeCount = in.readVLong(); this.forwardSmallSeeks = new Counter(in); @@ -216,7 +230,9 @@ public static CacheIndexInputStats combine(CacheIndexInputStats cis1, CacheIndex return new CacheIndexInputStats( cis1.fileExt, cis1.numFiles + cis2.numFiles, - cis1.totalSize + cis2.totalSize, + new ByteSizeValue(Math.addExact(cis1.totalSize.getBytes(), cis2.totalSize.getBytes())), + new ByteSizeValue(Math.min(cis1.minSize.getBytes(), cis2.minSize.getBytes())), + new ByteSizeValue(Math.max(cis1.maxSize.getBytes(), cis2.maxSize.getBytes())), cis1.openCount + cis2.openCount, cis1.closeCount + cis2.closeCount, cis1.forwardSmallSeeks.add(cis2.forwardSmallSeeks), @@ -245,7 +261,13 @@ public void writeTo(StreamOutput out) throws IOException { } out.writeString(fileExt); out.writeVLong(numFiles); - out.writeVLong(totalSize); + if (out.getVersion().before(Version.V_7_13_0)) { + out.writeVLong(totalSize.getBytes()); + } else { + totalSize.writeTo(out); + minSize.writeTo(out); + maxSize.writeTo(out); + } out.writeVLong(openCount); out.writeVLong(closeCount); @@ -276,10 +298,23 @@ public long getNumFiles() { return numFiles; } - public long getTotalSize() { + public ByteSizeValue getTotalSize() { return totalSize; } + public ByteSizeValue getMinSize() { + return minSize; + } + + public ByteSizeValue getMaxSize() { + return maxSize; + } + + public ByteSizeValue getAverageSize() { + final double average = (double) totalSize.getBytes()/ (double) numFiles; + return new ByteSizeValue(Math.round(average)); + } + public long getOpenCount() { return openCount; } @@ -346,9 +381,16 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws { builder.field("file_ext", getFileExt()); builder.field("num_files", getNumFiles()); - builder.field("total_size", getTotalSize()); builder.field("open_count", getOpenCount()); builder.field("close_count", getCloseCount()); + { + builder.startObject("size"); + builder.humanReadableField("total_in_bytes", "total", getTotalSize()); + builder.humanReadableField("min_in_bytes", "min", getMinSize()); + builder.humanReadableField("max_in_bytes", "max", getMaxSize()); + builder.humanReadableField("average_in_bytes", "average", getAverageSize()); + builder.endObject(); + } builder.field("contiguous_bytes_read", getContiguousReads(), params); builder.field("non_contiguous_bytes_read", getNonContiguousReads(), params); builder.field("cached_bytes_read", getCachedBytesRead(), params); @@ -384,10 +426,12 @@ public boolean equals(Object other) { } CacheIndexInputStats stats = (CacheIndexInputStats) other; return numFiles == stats.numFiles - && totalSize == stats.totalSize && openCount == stats.openCount && closeCount == stats.closeCount && Objects.equals(fileExt, stats.fileExt) + && Objects.equals(totalSize, stats.totalSize) + && Objects.equals(minSize, stats.minSize) + && Objects.equals(maxSize, stats.maxSize) && Objects.equals(forwardSmallSeeks, stats.forwardSmallSeeks) && Objects.equals(backwardSmallSeeks, stats.backwardSmallSeeks) && Objects.equals(forwardLargeSeeks, stats.forwardLargeSeeks) @@ -405,7 +449,9 @@ public boolean equals(Object other) { @Override public int hashCode() { - return Objects.hash(fileExt, numFiles, totalSize, openCount, closeCount, + return Objects.hash(fileExt, numFiles, totalSize, + minSize, maxSize, + openCount, closeCount, forwardSmallSeeks, backwardSmallSeeks, forwardLargeSeeks, backwardLargeSeeks, contiguousReads, nonContiguousReads, diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/searchablesnapshots/SearchableSnapshotShardStatsTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/searchablesnapshots/SearchableSnapshotShardStatsTests.java index 4da38949b304e..358697ce080de 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/searchablesnapshots/SearchableSnapshotShardStatsTests.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/searchablesnapshots/SearchableSnapshotShardStatsTests.java @@ -10,6 +10,7 @@ import org.elasticsearch.cluster.routing.ShardRoutingState; import org.elasticsearch.cluster.routing.TestShardRouting; import org.elasticsearch.common.io.stream.Writeable; +import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.repositories.IndexId; import org.elasticsearch.snapshots.SnapshotId; import org.elasticsearch.test.AbstractWireSerializingTestCase; @@ -42,7 +43,8 @@ protected SearchableSnapshotShardStats createTestInstance() { } private CacheIndexInputStats randomCacheIndexInputStats() { - return new CacheIndexInputStats(randomAlphaOfLength(10), randomNonNegativeLong(), randomNonNegativeLong(), + return new CacheIndexInputStats(randomAlphaOfLength(10), randomNonNegativeLong(), new ByteSizeValue(randomNonNegativeLong()), + new ByteSizeValue(randomNonNegativeLong()), new ByteSizeValue(randomNonNegativeLong()), randomNonNegativeLong(), randomNonNegativeLong(), randomCounter(), randomCounter(), randomCounter(), randomCounter(), diff --git a/x-pack/plugin/searchable-snapshots/qa/rest/src/test/resources/rest-api-spec/test/stats.yml b/x-pack/plugin/searchable-snapshots/qa/rest/src/test/resources/rest-api-spec/test/stats.yml index b7b492025b036..7c366ec532bd7 100644 --- a/x-pack/plugin/searchable-snapshots/qa/rest/src/test/resources/rest-api-spec/test/stats.yml +++ b/x-pack/plugin/searchable-snapshots/qa/rest/src/test/resources/rest-api-spec/test/stats.yml @@ -66,8 +66,8 @@ teardown: --- "Tests searchable snapshots stats": - skip: - version: " - 7.7.99" - reason: searchable snapshots introduced in 7.8.0 + version: " - 7.12.99" + reason: searchable snapshots response format changed in 7.13.0 - do: catch: missing @@ -139,10 +139,14 @@ teardown: - is_true: total - is_true: total.0.file_ext - gt: { total.0.num_files: 0 } - - gt: { total.0.total_size: 0 } - gt: { total.0.open_count: 0 } - gt: { total.0.close_count: 0 } + - gt: { total.0.size.total_in_bytes: 0 } + - gt: { total.0.size.min_in_bytes: 0 } + - gt: { total.0.size.max_in_bytes: 0 } + - gt: { total.0.size.average_in_bytes: 0 } + - gte: { total.0.contiguous_bytes_read.count: 0 } - gte: { total.0.contiguous_bytes_read.sum: 0 } - gte: { total.0.contiguous_bytes_read.min: 0 } @@ -215,10 +219,14 @@ teardown: - is_true: indices.docs.total.0.file_ext - gt: { indices.docs.total.0.num_files: 0 } - - gt: { indices.docs.total.0.total_size: 0 } - gt: { indices.docs.total.0.open_count: 0 } - gt: { indices.docs.total.0.close_count: 0 } + - gt: { indices.docs.total.0.size.total_in_bytes: 0 } + - gt: { indices.docs.total.0.size.min_in_bytes: 0 } + - gt: { indices.docs.total.0.size.max_in_bytes: 0 } + - gt: { indices.docs.total.0.size.average_in_bytes: 0 } + - gte: { indices.docs.total.0.contiguous_bytes_read.count: 0 } - gte: { indices.docs.total.0.contiguous_bytes_read.sum: 0 } - gte: { indices.docs.total.0.contiguous_bytes_read.min: 0 } @@ -295,10 +303,14 @@ teardown: - is_true: indices.docs.shards.0.0.files.0.file_ext - gt: { indices.docs.shards.0.0.files.0.num_files: 0 } - - gt: { indices.docs.shards.0.0.files.0.total_size: 0 } - gt: { indices.docs.shards.0.0.files.0.open_count: 0 } - gt: { indices.docs.shards.0.0.files.0.close_count: 0 } + - gt: { indices.docs.shards.0.0.files.0.size.total_in_bytes: 0 } + - gt: { indices.docs.shards.0.0.files.0.size.min_in_bytes: 0 } + - gt: { indices.docs.shards.0.0.files.0.size.max_in_bytes: 0 } + - gt: { indices.docs.shards.0.0.files.0.size.average_in_bytes: 0 } + - gte: { indices.docs.shards.0.0.files.0.contiguous_bytes_read.count: 0 } - gte: { indices.docs.shards.0.0.files.0.contiguous_bytes_read.sum: 0 } - gte: { indices.docs.shards.0.0.files.0.contiguous_bytes_read.min: 0 } @@ -373,6 +385,10 @@ teardown: - is_true: indices.docs.shards.0.0.files.0.cached_bytes_written.time - is_true: indices.docs.shards.0.0.files.0.direct_bytes_read.time + - is_true: indices.docs.shards.0.0.files.0.size.total + - is_true: indices.docs.shards.0.0.files.0.size.min + - is_true: indices.docs.shards.0.0.files.0.size.max + - is_true: indices.docs.shards.0.0.files.0.size.average - do: searchable_snapshots.stats: diff --git a/x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/SearchableSnapshotsIntegTests.java b/x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/SearchableSnapshotsIntegTests.java index 056f7aefffaac..ae3f9c0edd4e3 100644 --- a/x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/SearchableSnapshotsIntegTests.java +++ b/x-pack/plugin/searchable-snapshots/src/internalClusterTest/java/org/elasticsearch/xpack/searchablesnapshots/SearchableSnapshotsIntegTests.java @@ -1446,7 +1446,7 @@ private void assertSearchableSnapshotStats(String indexName, boolean cacheEnable final long totalSize = statsResponse.getStats() .stream() .flatMap(s -> s.getStats().stream()) - .mapToLong(SearchableSnapshotShardStats.CacheIndexInputStats::getTotalSize) + .mapToLong(stat -> stat.getTotalSize().getBytes()) .sum(); final Set nodeIdsWithLargeEnoughCache = new HashSet<>(); for (ObjectCursor nodeCursor : client().admin() @@ -1484,7 +1484,22 @@ private void assertSearchableSnapshotStats(String indexName, boolean cacheEnable ); assertThat( "Unexpected file length for " + fileExt + " of shard " + shardRouting, - indexInputStats.getTotalSize(), + indexInputStats.getTotalSize().getBytes(), + greaterThan(0L) + ); + assertThat( + "Unexpected min. file length for " + fileExt + " of shard " + shardRouting, + indexInputStats.getMinSize().getBytes(), + greaterThan(0L) + ); + assertThat( + "Unexpected max. file length for " + fileExt + " of shard " + shardRouting, + indexInputStats.getMaxSize().getBytes(), + greaterThan(0L) + ); + assertThat( + "Unexpected average file length for " + fileExt + " of shard " + shardRouting, + indexInputStats.getAverageSize().getBytes(), greaterThan(0L) ); diff --git a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/IndexInputStats.java b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/IndexInputStats.java index dc168c0b5df77..58f3956aa7f40 100644 --- a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/IndexInputStats.java +++ b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/IndexInputStats.java @@ -25,8 +25,11 @@ public class IndexInputStats { /* A threshold beyond which an index input seeking is counted as "large" */ static final ByteSizeValue SEEKING_THRESHOLD = new ByteSizeValue(8, ByteSizeUnit.MB); - private final int numFiles; + private final long numFiles; private final long totalSize; + private final long minSize; + private final long maxSize; + private final long seekingThreshold; private final LongSupplier currentTimeNanos; @@ -52,13 +55,22 @@ public class IndexInputStats { private final Counter blobStoreBytesRequested = new Counter(); private final AtomicLong currentIndexCacheFills = new AtomicLong(); - public IndexInputStats(int numFiles, long totalSize, LongSupplier currentTimeNanos) { - this(numFiles, totalSize, SEEKING_THRESHOLD.getBytes(), currentTimeNanos); + public IndexInputStats(long numFiles, long totalSize, long minSize, long maxSize, LongSupplier currentTimeNanos) { + this(numFiles, totalSize, minSize, maxSize, SEEKING_THRESHOLD.getBytes(), currentTimeNanos); } - public IndexInputStats(int numFiles, long totalSize, long seekingThreshold, LongSupplier currentTimeNanos) { + public IndexInputStats( + long numFiles, + long totalSize, + long minSize, + long maxSize, + long seekingThreshold, + LongSupplier currentTimeNanos + ) { this.numFiles = numFiles; this.totalSize = totalSize; + this.minSize = minSize; + this.maxSize = maxSize; this.seekingThreshold = seekingThreshold; this.currentTimeNanos = currentTimeNanos; } @@ -137,7 +149,7 @@ public Releasable addIndexCacheFill() { }; } - public int getNumFiles() { + public long getNumFiles() { return numFiles; } @@ -145,6 +157,14 @@ public long getTotalSize() { return totalSize; } + public long getMinSize() { + return minSize; + } + + public long getMaxSize() { + return maxSize; + } + public LongAdder getOpened() { return opened; } diff --git a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/SearchableSnapshotDirectory.java b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/SearchableSnapshotDirectory.java index 7d559e24073f9..f4f0d200d689b 100644 --- a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/SearchableSnapshotDirectory.java +++ b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/index/store/SearchableSnapshotDirectory.java @@ -360,8 +360,8 @@ public void clearCache() { } } - protected IndexInputStats createIndexInputStats(final int numFiles, final long totalSize) { - return new IndexInputStats(numFiles, totalSize, statsCurrentTimeNanosSupplier); + protected IndexInputStats createIndexInputStats(long numFiles, long totalSize, long minSize, long maxSize) { + return new IndexInputStats(numFiles, totalSize, minSize, maxSize, statsCurrentTimeNanosSupplier); } public CacheKey createCacheKey(String fileName) { @@ -395,13 +395,13 @@ public IndexInput openInput(final String name, final IOContext context) throws I final String ext = getNonNullFileExt(name); final IndexInputStats inputStats = stats.computeIfAbsent(ext, n -> { - // get all fileInfo with same extension - final Tuple fileExtCompoundStats = files().stream() - .filter(fi -> ext.equals(getNonNullFileExt(fi.physicalName()))) - .map(fi -> Tuple.tuple(1, fi.length())) - .reduce((t1, t2) -> Tuple.tuple(t1.v1() + t2.v1(), t1.v2() + t2.v2())) - .get(); - return createIndexInputStats(fileExtCompoundStats.v1(), fileExtCompoundStats.v2()); + final IndexInputStats.Counter counter = new IndexInputStats.Counter(); + for (BlobStoreIndexShardSnapshot.FileInfo file : files()) { + if (ext.equals(getNonNullFileExt(file.physicalName()))) { + counter.add(file.length()); + } + } + return createIndexInputStats(counter.count(), counter.total(), counter.min(), counter.max()); }); if (useCache && isExcludedFromCache(name) == false) { if (partial) { diff --git a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/xpack/searchablesnapshots/action/SearchableSnapshotsStatsResponse.java b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/xpack/searchablesnapshots/action/SearchableSnapshotsStatsResponse.java index 2f44cc6f7c9b3..bf3a1ed941064 100644 --- a/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/xpack/searchablesnapshots/action/SearchableSnapshotsStatsResponse.java +++ b/x-pack/plugin/searchable-snapshots/src/main/java/org/elasticsearch/xpack/searchablesnapshots/action/SearchableSnapshotsStatsResponse.java @@ -65,6 +65,7 @@ private static List computeCompound(Stream { + + final Map numFiles = new HashMap<>(); + final Map totalFiles = new HashMap<>(); + final Map minSizes = new HashMap<>(); + final Map maxSizes = new HashMap<>(); + + final Predicate ignoredExtensions = fileExtension -> "lock".equals(fileExtension) + || "si".equals(fileExtension) + || fileExtension.isEmpty(); + + for (String fileName : directory.listAll()) { + final String extension = getNonNullFileExt(fileName); + if (ignoredExtensions.test(extension)) { + continue; + } + + // open each file once to force the creation of index input stats + try (IndexInput input = snapshotDirectory.openInput(fileName, randomIOContext())) { + numFiles.compute(extension, (ext, num) -> num == null ? 1L : num + 1L); + totalFiles.compute(extension, (ext, total) -> total != null ? total + input.length() : input.length()); + minSizes.compute(extension, (ext, min) -> Math.min(input.length(), min != null ? min : Long.MAX_VALUE)); + maxSizes.compute(extension, (ext, max) -> Math.max(input.length(), max != null ? max : Long.MIN_VALUE)); + } + } + + for (String fileName : snapshotDirectory.listAll()) { + final String extension = getNonNullFileExt(fileName); + if (ignoredExtensions.test(extension)) { + continue; + } + + final IndexInputStats inputStats = snapshotDirectory.getStats(fileName); + assertThat("No index input stats for extension [" + extension + ']', inputStats, notNullValue()); + assertThat(inputStats.getNumFiles(), equalTo(numFiles.get(extension))); + assertThat(inputStats.getTotalSize(), equalTo(totalFiles.get(extension))); + assertThat(inputStats.getMinSize(), equalTo(minSizes.get(extension))); + assertThat(inputStats.getMaxSize(), equalTo(maxSizes.get(extension))); + } + }); + } + public void testIndexSearcher() throws Exception { testDirectories((directory, snapshotDirectory) -> { try (DirectoryReader reader = DirectoryReader.open(directory)) { diff --git a/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/index/store/direct/DirectBlobContainerIndexInputTests.java b/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/index/store/direct/DirectBlobContainerIndexInputTests.java index f44a345b76bfb..91a5d098298e9 100644 --- a/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/index/store/direct/DirectBlobContainerIndexInputTests.java +++ b/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/index/store/direct/DirectBlobContainerIndexInputTests.java @@ -132,7 +132,7 @@ public int read(byte[] b, int off, int len) throws IOException { directory, fileInfo, randomIOContext(), - new IndexInputStats(1, 0L, () -> 0L), + new IndexInputStats(1L, fileInfo.length(), fileInfo.length(), fileInfo.length(), () -> 0L), minimumReadSize, randomBoolean() ? BufferedIndexInput.BUFFER_SIZE : between(BufferedIndexInput.MIN_BUFFER_SIZE, BufferedIndexInput.BUFFER_SIZE) ); diff --git a/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/xpack/searchablesnapshots/action/SearchableSnapshotsStatsResponseTests.java b/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/xpack/searchablesnapshots/action/SearchableSnapshotsStatsResponseTests.java index 67134ba29b1a7..465090023328d 100644 --- a/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/xpack/searchablesnapshots/action/SearchableSnapshotsStatsResponseTests.java +++ b/x-pack/plugin/searchable-snapshots/src/test/java/org/elasticsearch/xpack/searchablesnapshots/action/SearchableSnapshotsStatsResponseTests.java @@ -11,6 +11,7 @@ import org.elasticsearch.action.support.DefaultShardOperationFailedException; import org.elasticsearch.cluster.routing.ShardRouting; import org.elasticsearch.cluster.routing.ShardRoutingState; +import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.repositories.IndexId; import org.elasticsearch.snapshots.SnapshotId; import org.elasticsearch.test.ESTestCase; @@ -99,7 +100,9 @@ private static SearchableSnapshotShardStats.CacheIndexInputStats randomCacheInde return new SearchableSnapshotShardStats.CacheIndexInputStats( randomAlphaOfLength(10), randomNonNegativeLong(), - randomNonNegativeLong(), + new ByteSizeValue(randomNonNegativeLong()), + new ByteSizeValue(randomNonNegativeLong()), + new ByteSizeValue(randomNonNegativeLong()), randomNonNegativeLong(), randomNonNegativeLong(), randomCounter(),