[ 
https://issues.apache.org/jira/browse/IGNITE-23316?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Vladislav Pyatkov updated IGNITE-23316:
---------------------------------------
    Description: 
h3. Motivation
Thread-safe update is an expensive operation, and we ought to avoid it in cases 
where it is not needed. The stack below shows the usage update operation where 
the result value in not used:
{code}
if (response instanceof TimestampAware) {
    clock.update(((TimestampAware) response).timestamp());
}
{code}
{noformat}
hybridClock:update-:java.lang.Exception: 
        at 
org.apache.ignite.internal.util.IgniteUtils.dumpStack(IgniteUtils.java:682)
        at 
org.apache.ignite.internal.hlc.HybridClockImpl.update(HybridClockImpl.java:119)
        at 
org.apache.ignite.internal.replicator.ReplicaService.lambda$sendToReplica$7(ReplicaService.java:164)
        at 
java.base/java.util.concurrent.CompletableFuture.uniWhenComplete(CompletableFuture.java:859)
        at 
java.base/java.util.concurrent.CompletableFuture.uniWhenCompleteStage(CompletableFuture.java:883)
        at 
java.base/java.util.concurrent.CompletableFuture.whenComplete(CompletableFuture.java:2257)
        at 
org.apache.ignite.internal.replicator.ReplicaService.sendToReplica(ReplicaService.java:144)
        at 
org.apache.ignite.internal.replicator.ReplicaService.invoke(ReplicaService.java:275)
        at 
org.apache.ignite.internal.table.distributed.storage.InternalTableImpl.lambda$evaluateReadOnlyPrimaryNode$19(InternalTableImpl.java:772)
        at 
java.base/java.util.concurrent.CompletableFuture.uniComposeStage(CompletableFuture.java:1106)
        at 
java.base/java.util.concurrent.CompletableFuture.thenCompose(CompletableFuture.java:2241)
        at 
org.apache.ignite.internal.table.distributed.storage.InternalTableImpl.evaluateReadOnlyPrimaryNode(InternalTableImpl.java:763)
        at 
org.apache.ignite.internal.table.distributed.storage.InternalTableImpl.get(InternalTableImpl.java:859)
        at 
org.apache.ignite.internal.table.KeyValueBinaryViewImpl.lambda$getAsync$1(KeyValueBinaryViewImpl.java:120)
        at 
org.apache.ignite.internal.table.AbstractTableView.lambda$withSchemaSync$2(AbstractTableView.java:143)
        at 
java.base/java.util.concurrent.CompletableFuture.uniComposeStage(CompletableFuture.java:1106)
        at 
java.base/java.util.concurrent.CompletableFuture.thenCompose(CompletableFuture.java:2241)
        at 
org.apache.ignite.internal.table.AbstractTableView.withSchemaSync(AbstractTableView.java:143)
        at 
org.apache.ignite.internal.table.AbstractTableView.withSchemaSync(AbstractTableView.java:134)
        at 
org.apache.ignite.internal.table.AbstractTableView.doOperation(AbstractTableView.java:112)
        at 
org.apache.ignite.internal.table.KeyValueBinaryViewImpl.getAsync(KeyValueBinaryViewImpl.java:117)
        at 
org.apache.ignite.internal.table.KeyValueBinaryViewImpl.get(KeyValueBinaryViewImpl.java:104)
        at 
org.apache.ignite.internal.table.KeyValueBinaryViewImpl.get(KeyValueBinaryViewImpl.java:74)
        at 
org.apache.ignite.internal.table.PublicApiThreadingKeyValueView.lambda$get$0(PublicApiThreadingKeyValueView.java:57)
        at 
org.apache.ignite.internal.thread.PublicApiThreading.executeWithRole(PublicApiThreading.java:144)
        at 
org.apache.ignite.internal.thread.PublicApiThreading.execUserSyncOperation(PublicApiThreading.java:102)
        at 
org.apache.ignite.internal.table.PublicApiThreadingViewBase.executeSyncOp(PublicApiThreadingViewBase.java:107)
        at 
org.apache.ignite.internal.table.PublicApiThreadingKeyValueView.get(PublicApiThreadingKeyValueView.java:57)
        at 
org.apache.ignite.internal.restart.RestartProofKeyValueView.lambda$get$0(RestartProofKeyValueView.java:58)
        at 
org.apache.ignite.internal.restart.RestartProofApiObject.lambda$attached$0(RestartProofApiObject.java:46)
        at 
org.apache.ignite.internal.restart.IgniteAttachmentLock.attached(IgniteAttachmentLock.java:59)
        at 
org.apache.ignite.internal.restart.RestartProofApiObject.attached(RestartProofApiObject.java:46)
        at 
org.apache.ignite.internal.restart.RestartProofKeyValueView.get(RestartProofKeyValueView.java:58)
        at 
org.apache.ignite.internal.benchmark.SelectBenchmark.kvGet(SelectBenchmark.java:175)
        at 
org.apache.ignite.internal.benchmark.jmh_generated.SelectBenchmark_kvGet_jmhTest.kvGet_avgt_jmhStub(SelectBenchmark_kvGet_jmhTest.java:238)
        at 
org.apache.ignite.internal.benchmark.jmh_generated.SelectBenchmark_kvGet_jmhTest.kvGet_AverageTime(SelectBenchmark_kvGet_jmhTest.java:177)
        at jdk.internal.reflect.GeneratedMethodAccessor17.invoke(Unknown Source)
        at 
java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
        at java.base/java.lang.reflect.Method.invoke(Method.java:566)
        at 
org.openjdk.jmh.runner.BenchmarkHandler$BenchmarkTask.call(BenchmarkHandler.java:527)
        at 
org.openjdk.jmh.runner.BenchmarkHandler$BenchmarkTask.call(BenchmarkHandler.java:504)
        at java.base/java.util.concurrent.FutureTask.run(FutureTask.java:264)
        at 
java.base/java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:515)
        at java.base/java.util.concurrent.FutureTask.run(FutureTask.java:264)
        at 
java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)
        at 
java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)
        at java.base/java.lang.Thread.run(Thread.java:829)
{noformat}

h3. Implementation notes
My proposal of the fast update operation:
{code}
  public void fastUpdate(HybridTimestamp requestTime) {
        long requestTimeLong = requestTime.longValue();

        while (true) {
            long now = currentTime();

            if (requestTimeLong <= now) {
                return;
            }

            // Read the latest time after accessing UTC time to reduce 
contention.
            long oldLatestTime = this.latestTime;

            if (requestTimeLong <= oldLatestTime) {
                return;
            }

            if (LATEST_TIME.compareAndSet(this, oldLatestTime, 
requestTimeLong)) {
                notifyUpdateListeners(requestTimeLong);

                return;
            }
        }
    }
{code}

h3. Definition of done
Replace all HybridClock#update's to the fastUpdate where the result value is 
not used.


  was:
h3. Motivation
Thread-safe update is an expensive operation, and we ought to avoid it in cases 
where it is not needed. The stack below shows the usage update operation where 
the result value in not used:
{code}
if (response instanceof TimestampAware) {
    clock.fastUpdate(((TimestampAware) response).timestamp());
}
{code}
{noformat}
hybridClock:update-:java.lang.Exception: 
        at 
org.apache.ignite.internal.util.IgniteUtils.dumpStack(IgniteUtils.java:682)
        at 
org.apache.ignite.internal.hlc.HybridClockImpl.update(HybridClockImpl.java:119)
        at 
org.apache.ignite.internal.replicator.ReplicaService.lambda$sendToReplica$7(ReplicaService.java:164)
        at 
java.base/java.util.concurrent.CompletableFuture.uniWhenComplete(CompletableFuture.java:859)
        at 
java.base/java.util.concurrent.CompletableFuture.uniWhenCompleteStage(CompletableFuture.java:883)
        at 
java.base/java.util.concurrent.CompletableFuture.whenComplete(CompletableFuture.java:2257)
        at 
org.apache.ignite.internal.replicator.ReplicaService.sendToReplica(ReplicaService.java:144)
        at 
org.apache.ignite.internal.replicator.ReplicaService.invoke(ReplicaService.java:275)
        at 
org.apache.ignite.internal.table.distributed.storage.InternalTableImpl.lambda$evaluateReadOnlyPrimaryNode$19(InternalTableImpl.java:772)
        at 
java.base/java.util.concurrent.CompletableFuture.uniComposeStage(CompletableFuture.java:1106)
        at 
java.base/java.util.concurrent.CompletableFuture.thenCompose(CompletableFuture.java:2241)
        at 
org.apache.ignite.internal.table.distributed.storage.InternalTableImpl.evaluateReadOnlyPrimaryNode(InternalTableImpl.java:763)
        at 
org.apache.ignite.internal.table.distributed.storage.InternalTableImpl.get(InternalTableImpl.java:859)
        at 
org.apache.ignite.internal.table.KeyValueBinaryViewImpl.lambda$getAsync$1(KeyValueBinaryViewImpl.java:120)
        at 
org.apache.ignite.internal.table.AbstractTableView.lambda$withSchemaSync$2(AbstractTableView.java:143)
        at 
java.base/java.util.concurrent.CompletableFuture.uniComposeStage(CompletableFuture.java:1106)
        at 
java.base/java.util.concurrent.CompletableFuture.thenCompose(CompletableFuture.java:2241)
        at 
org.apache.ignite.internal.table.AbstractTableView.withSchemaSync(AbstractTableView.java:143)
        at 
org.apache.ignite.internal.table.AbstractTableView.withSchemaSync(AbstractTableView.java:134)
        at 
org.apache.ignite.internal.table.AbstractTableView.doOperation(AbstractTableView.java:112)
        at 
org.apache.ignite.internal.table.KeyValueBinaryViewImpl.getAsync(KeyValueBinaryViewImpl.java:117)
        at 
org.apache.ignite.internal.table.KeyValueBinaryViewImpl.get(KeyValueBinaryViewImpl.java:104)
        at 
org.apache.ignite.internal.table.KeyValueBinaryViewImpl.get(KeyValueBinaryViewImpl.java:74)
        at 
org.apache.ignite.internal.table.PublicApiThreadingKeyValueView.lambda$get$0(PublicApiThreadingKeyValueView.java:57)
        at 
org.apache.ignite.internal.thread.PublicApiThreading.executeWithRole(PublicApiThreading.java:144)
        at 
org.apache.ignite.internal.thread.PublicApiThreading.execUserSyncOperation(PublicApiThreading.java:102)
        at 
org.apache.ignite.internal.table.PublicApiThreadingViewBase.executeSyncOp(PublicApiThreadingViewBase.java:107)
        at 
org.apache.ignite.internal.table.PublicApiThreadingKeyValueView.get(PublicApiThreadingKeyValueView.java:57)
        at 
org.apache.ignite.internal.restart.RestartProofKeyValueView.lambda$get$0(RestartProofKeyValueView.java:58)
        at 
org.apache.ignite.internal.restart.RestartProofApiObject.lambda$attached$0(RestartProofApiObject.java:46)
        at 
org.apache.ignite.internal.restart.IgniteAttachmentLock.attached(IgniteAttachmentLock.java:59)
        at 
org.apache.ignite.internal.restart.RestartProofApiObject.attached(RestartProofApiObject.java:46)
        at 
org.apache.ignite.internal.restart.RestartProofKeyValueView.get(RestartProofKeyValueView.java:58)
        at 
org.apache.ignite.internal.benchmark.SelectBenchmark.kvGet(SelectBenchmark.java:175)
        at 
org.apache.ignite.internal.benchmark.jmh_generated.SelectBenchmark_kvGet_jmhTest.kvGet_avgt_jmhStub(SelectBenchmark_kvGet_jmhTest.java:238)
        at 
org.apache.ignite.internal.benchmark.jmh_generated.SelectBenchmark_kvGet_jmhTest.kvGet_AverageTime(SelectBenchmark_kvGet_jmhTest.java:177)
        at jdk.internal.reflect.GeneratedMethodAccessor17.invoke(Unknown Source)
        at 
java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
        at java.base/java.lang.reflect.Method.invoke(Method.java:566)
        at 
org.openjdk.jmh.runner.BenchmarkHandler$BenchmarkTask.call(BenchmarkHandler.java:527)
        at 
org.openjdk.jmh.runner.BenchmarkHandler$BenchmarkTask.call(BenchmarkHandler.java:504)
        at java.base/java.util.concurrent.FutureTask.run(FutureTask.java:264)
        at 
java.base/java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:515)
        at java.base/java.util.concurrent.FutureTask.run(FutureTask.java:264)
        at 
java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)
        at 
java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)
        at java.base/java.lang.Thread.run(Thread.java:829)
{noformat}


> Hybrid Clock#update can be optimized in case where return value is not needed
> -----------------------------------------------------------------------------
>
>                 Key: IGNITE-23316
>                 URL: https://issues.apache.org/jira/browse/IGNITE-23316
>             Project: Ignite
>          Issue Type: Improvement
>            Reporter: Vladislav Pyatkov
>            Priority: Major
>              Labels: ignite-3
>
> h3. Motivation
> Thread-safe update is an expensive operation, and we ought to avoid it in 
> cases where it is not needed. The stack below shows the usage update 
> operation where the result value in not used:
> {code}
> if (response instanceof TimestampAware) {
>     clock.update(((TimestampAware) response).timestamp());
> }
> {code}
> {noformat}
> hybridClock:update-:java.lang.Exception: 
>       at 
> org.apache.ignite.internal.util.IgniteUtils.dumpStack(IgniteUtils.java:682)
>       at 
> org.apache.ignite.internal.hlc.HybridClockImpl.update(HybridClockImpl.java:119)
>       at 
> org.apache.ignite.internal.replicator.ReplicaService.lambda$sendToReplica$7(ReplicaService.java:164)
>       at 
> java.base/java.util.concurrent.CompletableFuture.uniWhenComplete(CompletableFuture.java:859)
>       at 
> java.base/java.util.concurrent.CompletableFuture.uniWhenCompleteStage(CompletableFuture.java:883)
>       at 
> java.base/java.util.concurrent.CompletableFuture.whenComplete(CompletableFuture.java:2257)
>       at 
> org.apache.ignite.internal.replicator.ReplicaService.sendToReplica(ReplicaService.java:144)
>       at 
> org.apache.ignite.internal.replicator.ReplicaService.invoke(ReplicaService.java:275)
>       at 
> org.apache.ignite.internal.table.distributed.storage.InternalTableImpl.lambda$evaluateReadOnlyPrimaryNode$19(InternalTableImpl.java:772)
>       at 
> java.base/java.util.concurrent.CompletableFuture.uniComposeStage(CompletableFuture.java:1106)
>       at 
> java.base/java.util.concurrent.CompletableFuture.thenCompose(CompletableFuture.java:2241)
>       at 
> org.apache.ignite.internal.table.distributed.storage.InternalTableImpl.evaluateReadOnlyPrimaryNode(InternalTableImpl.java:763)
>       at 
> org.apache.ignite.internal.table.distributed.storage.InternalTableImpl.get(InternalTableImpl.java:859)
>       at 
> org.apache.ignite.internal.table.KeyValueBinaryViewImpl.lambda$getAsync$1(KeyValueBinaryViewImpl.java:120)
>       at 
> org.apache.ignite.internal.table.AbstractTableView.lambda$withSchemaSync$2(AbstractTableView.java:143)
>       at 
> java.base/java.util.concurrent.CompletableFuture.uniComposeStage(CompletableFuture.java:1106)
>       at 
> java.base/java.util.concurrent.CompletableFuture.thenCompose(CompletableFuture.java:2241)
>       at 
> org.apache.ignite.internal.table.AbstractTableView.withSchemaSync(AbstractTableView.java:143)
>       at 
> org.apache.ignite.internal.table.AbstractTableView.withSchemaSync(AbstractTableView.java:134)
>       at 
> org.apache.ignite.internal.table.AbstractTableView.doOperation(AbstractTableView.java:112)
>       at 
> org.apache.ignite.internal.table.KeyValueBinaryViewImpl.getAsync(KeyValueBinaryViewImpl.java:117)
>       at 
> org.apache.ignite.internal.table.KeyValueBinaryViewImpl.get(KeyValueBinaryViewImpl.java:104)
>       at 
> org.apache.ignite.internal.table.KeyValueBinaryViewImpl.get(KeyValueBinaryViewImpl.java:74)
>       at 
> org.apache.ignite.internal.table.PublicApiThreadingKeyValueView.lambda$get$0(PublicApiThreadingKeyValueView.java:57)
>       at 
> org.apache.ignite.internal.thread.PublicApiThreading.executeWithRole(PublicApiThreading.java:144)
>       at 
> org.apache.ignite.internal.thread.PublicApiThreading.execUserSyncOperation(PublicApiThreading.java:102)
>       at 
> org.apache.ignite.internal.table.PublicApiThreadingViewBase.executeSyncOp(PublicApiThreadingViewBase.java:107)
>       at 
> org.apache.ignite.internal.table.PublicApiThreadingKeyValueView.get(PublicApiThreadingKeyValueView.java:57)
>       at 
> org.apache.ignite.internal.restart.RestartProofKeyValueView.lambda$get$0(RestartProofKeyValueView.java:58)
>       at 
> org.apache.ignite.internal.restart.RestartProofApiObject.lambda$attached$0(RestartProofApiObject.java:46)
>       at 
> org.apache.ignite.internal.restart.IgniteAttachmentLock.attached(IgniteAttachmentLock.java:59)
>       at 
> org.apache.ignite.internal.restart.RestartProofApiObject.attached(RestartProofApiObject.java:46)
>       at 
> org.apache.ignite.internal.restart.RestartProofKeyValueView.get(RestartProofKeyValueView.java:58)
>       at 
> org.apache.ignite.internal.benchmark.SelectBenchmark.kvGet(SelectBenchmark.java:175)
>       at 
> org.apache.ignite.internal.benchmark.jmh_generated.SelectBenchmark_kvGet_jmhTest.kvGet_avgt_jmhStub(SelectBenchmark_kvGet_jmhTest.java:238)
>       at 
> org.apache.ignite.internal.benchmark.jmh_generated.SelectBenchmark_kvGet_jmhTest.kvGet_AverageTime(SelectBenchmark_kvGet_jmhTest.java:177)
>       at jdk.internal.reflect.GeneratedMethodAccessor17.invoke(Unknown Source)
>       at 
> java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
>       at java.base/java.lang.reflect.Method.invoke(Method.java:566)
>       at 
> org.openjdk.jmh.runner.BenchmarkHandler$BenchmarkTask.call(BenchmarkHandler.java:527)
>       at 
> org.openjdk.jmh.runner.BenchmarkHandler$BenchmarkTask.call(BenchmarkHandler.java:504)
>       at java.base/java.util.concurrent.FutureTask.run(FutureTask.java:264)
>       at 
> java.base/java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:515)
>       at java.base/java.util.concurrent.FutureTask.run(FutureTask.java:264)
>       at 
> java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)
>       at 
> java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)
>       at java.base/java.lang.Thread.run(Thread.java:829)
> {noformat}
> h3. Implementation notes
> My proposal of the fast update operation:
> {code}
>   public void fastUpdate(HybridTimestamp requestTime) {
>         long requestTimeLong = requestTime.longValue();
>         while (true) {
>             long now = currentTime();
>             if (requestTimeLong <= now) {
>                 return;
>             }
>             // Read the latest time after accessing UTC time to reduce 
> contention.
>             long oldLatestTime = this.latestTime;
>             if (requestTimeLong <= oldLatestTime) {
>                 return;
>             }
>             if (LATEST_TIME.compareAndSet(this, oldLatestTime, 
> requestTimeLong)) {
>                 notifyUpdateListeners(requestTimeLong);
>                 return;
>             }
>         }
>     }
> {code}
> h3. Definition of done
> Replace all HybridClock#update's to the fastUpdate where the result value is 
> not used.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

Reply via email to