From 7a0c319b23c93809fc4aab72917c9c1828e18f7c Mon Sep 17 00:00:00 2001 From: matthias Date: Wed, 1 Jan 2025 23:22:45 +0100 Subject: [PATCH 001/113] refactor: use junit timeout exceptions and remove time constrains from individual statements --- .../ignite/raft/jraft/core/ItNodeTest.java | 19 ++++++++----------- 1 file changed, 8 insertions(+), 11 deletions(-) diff --git a/modules/raft/src/integrationTest/java/org/apache/ignite/raft/jraft/core/ItNodeTest.java b/modules/raft/src/integrationTest/java/org/apache/ignite/raft/jraft/core/ItNodeTest.java index e0e0cee115b..d4f1c929ca5 100644 --- a/modules/raft/src/integrationTest/java/org/apache/ignite/raft/jraft/core/ItNodeTest.java +++ b/modules/raft/src/integrationTest/java/org/apache/ignite/raft/jraft/core/ItNodeTest.java @@ -164,6 +164,7 @@ import org.junit.jupiter.api.Disabled; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.TestInfo; +import org.junit.jupiter.api.Timeout; import org.junit.jupiter.api.extension.ExtendWith; /** @@ -2706,12 +2707,12 @@ public void testAppendEntriesWhenFollowerIsInErrorState() throws Exception { } @Test - @Disabled("https://issues.apache.org/jira/browse/IGNITE-21792") + @Timeout(value = 10_000, unit = TimeUnit.MILLISECONDS) //todo change to at least 20_000 to get the added timeout from before! public void testFollowerStartStopFollowing() throws Exception { // start five nodes List peers = TestUtils.generatePeers(testInfo, 5); - cluster = new TestCluster("unitest", dataPath, peers, ELECTION_TIMEOUT_MILLIS, testInfo); + cluster = new TestCluster("unittest", dataPath, peers, ELECTION_TIMEOUT_MILLIS, testInfo); for (TestPeer peer : peers) assertTrue(cluster.start(peer)); @@ -2726,8 +2727,7 @@ public void testFollowerStartStopFollowing() throws Exception { List firstFollowers = cluster.getFollowers(); assertEquals(4, firstFollowers.size()); for (Node node : firstFollowers) { - assertTrue( - waitForCondition(() -> ((MockStateMachine) node.getOptions().getFsm()).getOnStartFollowingTimes() == 1, 5_000)); + assertEquals(1, ((MockStateMachine) node.getOptions().getFsm()).getOnStartFollowingTimes()); assertEquals(0, ((MockStateMachine) node.getOptions().getFsm()).getOnStopFollowingTimes()); } @@ -2742,8 +2742,7 @@ public void testFollowerStartStopFollowing() throws Exception { List secondFollowers = cluster.getFollowers(); assertEquals(3, secondFollowers.size()); for (Node node : secondFollowers) { - assertTrue( - waitForCondition(() -> ((MockStateMachine) node.getOptions().getFsm()).getOnStartFollowingTimes() == 2, 5_000)); + assertEquals(2, ((MockStateMachine) node.getOptions().getFsm()).getOnStartFollowingTimes()); assertEquals(1, ((MockStateMachine) node.getOptions().getFsm()).getOnStopFollowingTimes()); } @@ -2760,14 +2759,12 @@ public void testFollowerStartStopFollowing() throws Exception { for (int i = 0; i < 3; i++) { Node follower = thirdFollowers.get(i); if (follower.getNodeId().getPeerId().equals(secondLeader.getNodeId().getPeerId())) { - assertTrue( - waitForCondition(() -> ((MockStateMachine) follower.getOptions().getFsm()).getOnStartFollowingTimes() == 2, 5_000)); - assertEquals(1, - ((MockStateMachine) follower.getOptions().getFsm()).getOnStopFollowingTimes()); + assertEquals(2, ((MockStateMachine) follower.getOptions().getFsm()).getOnStartFollowingTimes()); + assertEquals(1, ((MockStateMachine) follower.getOptions().getFsm()).getOnStopFollowingTimes()); continue; } - assertTrue(waitForCondition(() -> ((MockStateMachine) follower.getOptions().getFsm()).getOnStartFollowingTimes() == 3, 5_000)); + assertEquals(3, ((MockStateMachine) follower.getOptions().getFsm()).getOnStartFollowingTimes()); assertEquals(2, ((MockStateMachine) follower.getOptions().getFsm()).getOnStopFollowingTimes()); } From 64cc5c787da4c6dadbe924a122ef9fe520cc4d47 Mon Sep 17 00:00:00 2001 From: matthias Date: Thu, 2 Jan 2025 00:47:37 +0100 Subject: [PATCH 002/113] temp: add test action to test flakiness in a ci environment --- .github/workflows/flaky_detection.yml | 52 +++++++++++++++++++++++++++ 1 file changed, 52 insertions(+) create mode 100644 .github/workflows/flaky_detection.yml diff --git a/.github/workflows/flaky_detection.yml b/.github/workflows/flaky_detection.yml new file mode 100644 index 00000000000..bc674079fe6 --- /dev/null +++ b/.github/workflows/flaky_detection.yml @@ -0,0 +1,52 @@ +name: Flaky detection + +on: [workflow_dispatch] + +jobs: + build: + + runs-on: ubuntu-latest + permissions: + contents: read + + steps: + - uses: actions/checkout@v4 + - name: Set up JDK 17 + uses: actions/setup-java@v4 + with: + java-version: '17' + distribution: 'temurin' + # Configure Gradle for optimal use in GitHub Actions, including caching of downloaded dependencies. + # See: https://github.com/gradle/actions/blob/main/setup-gradle/README.md + - name: Setup Gradle + uses: gradle/actions/setup-gradle@af1da67850ed9a4cedd57bfd976089dd991e2582 # v4.0.0 + + - name: Test 1 + run: ./gradlew :ignite-raft:test + + - name: Test 2 + run: ./gradlew :ignite-raft:test + + - name: Test 3 + run: ./gradlew :ignite-raft:test + + - name: Test 4 + run: ./gradlew :ignite-raft:test + + - name: Test 5 + run: ./gradlew :ignite-raft:test + + - name: Test 6 + run: ./gradlew :ignite-raft:test + + - name: Test 7 + run: ./gradlew :ignite-raft:test + + - name: Test 8 + run: ./gradlew :ignite-raft:test + + - name: Test 9 + run: ./gradlew :ignite-raft:test + + - name: Test 10 + run: ./gradlew :ignite-raft:test From 74f54b743b70964b2ca6fe20ed4e3d9faec7308b Mon Sep 17 00:00:00 2001 From: matthias Date: Thu, 2 Jan 2025 01:07:15 +0100 Subject: [PATCH 003/113] remove flaky detection action and increase timeout just to be safe use the old timeout added together + 5 seconds --- .../java/org/apache/ignite/raft/jraft/core/ItNodeTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/modules/raft/src/integrationTest/java/org/apache/ignite/raft/jraft/core/ItNodeTest.java b/modules/raft/src/integrationTest/java/org/apache/ignite/raft/jraft/core/ItNodeTest.java index d4f1c929ca5..1446576666c 100644 --- a/modules/raft/src/integrationTest/java/org/apache/ignite/raft/jraft/core/ItNodeTest.java +++ b/modules/raft/src/integrationTest/java/org/apache/ignite/raft/jraft/core/ItNodeTest.java @@ -2707,7 +2707,7 @@ public void testAppendEntriesWhenFollowerIsInErrorState() throws Exception { } @Test - @Timeout(value = 10_000, unit = TimeUnit.MILLISECONDS) //todo change to at least 20_000 to get the added timeout from before! + @Timeout(value = 25_000, unit = TimeUnit.MILLISECONDS) public void testFollowerStartStopFollowing() throws Exception { // start five nodes List peers = TestUtils.generatePeers(testInfo, 5); From 88cd09394ead0c5c25b035efe25f9d1cf55a6f56 Mon Sep 17 00:00:00 2001 From: matthias Date: Thu, 2 Jan 2025 01:07:37 +0100 Subject: [PATCH 004/113] remove flaky detection action --- .github/workflows/flaky_detection.yml | 52 --------------------------- 1 file changed, 52 deletions(-) delete mode 100644 .github/workflows/flaky_detection.yml diff --git a/.github/workflows/flaky_detection.yml b/.github/workflows/flaky_detection.yml deleted file mode 100644 index bc674079fe6..00000000000 --- a/.github/workflows/flaky_detection.yml +++ /dev/null @@ -1,52 +0,0 @@ -name: Flaky detection - -on: [workflow_dispatch] - -jobs: - build: - - runs-on: ubuntu-latest - permissions: - contents: read - - steps: - - uses: actions/checkout@v4 - - name: Set up JDK 17 - uses: actions/setup-java@v4 - with: - java-version: '17' - distribution: 'temurin' - # Configure Gradle for optimal use in GitHub Actions, including caching of downloaded dependencies. - # See: https://github.com/gradle/actions/blob/main/setup-gradle/README.md - - name: Setup Gradle - uses: gradle/actions/setup-gradle@af1da67850ed9a4cedd57bfd976089dd991e2582 # v4.0.0 - - - name: Test 1 - run: ./gradlew :ignite-raft:test - - - name: Test 2 - run: ./gradlew :ignite-raft:test - - - name: Test 3 - run: ./gradlew :ignite-raft:test - - - name: Test 4 - run: ./gradlew :ignite-raft:test - - - name: Test 5 - run: ./gradlew :ignite-raft:test - - - name: Test 6 - run: ./gradlew :ignite-raft:test - - - name: Test 7 - run: ./gradlew :ignite-raft:test - - - name: Test 8 - run: ./gradlew :ignite-raft:test - - - name: Test 9 - run: ./gradlew :ignite-raft:test - - - name: Test 10 - run: ./gradlew :ignite-raft:test From 4f7f341c16f063a776a97ea835aec20ef0de2e0a Mon Sep 17 00:00:00 2001 From: Ivan Bessonov Date: Thu, 9 Jan 2025 15:29:14 +0300 Subject: [PATCH 005/113] IGNITE-23597 Cache latest term values in log manager (#5005) --- .../internal/raft/storage/TermCache.java | 140 ++++++++++++++++++ .../jraft/storage/impl/LogManagerImpl.java | 26 ++-- .../internal/raft/storage/TermCacheTest.java | 90 +++++++++++ 3 files changed, 247 insertions(+), 9 deletions(-) create mode 100644 modules/raft/src/main/java/org/apache/ignite/internal/raft/storage/TermCache.java create mode 100644 modules/raft/src/test/java/org/apache/ignite/internal/raft/storage/TermCacheTest.java diff --git a/modules/raft/src/main/java/org/apache/ignite/internal/raft/storage/TermCache.java b/modules/raft/src/main/java/org/apache/ignite/internal/raft/storage/TermCache.java new file mode 100644 index 00000000000..43eb4f58e0b --- /dev/null +++ b/modules/raft/src/main/java/org/apache/ignite/internal/raft/storage/TermCache.java @@ -0,0 +1,140 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.raft.storage; + +import static org.apache.ignite.internal.util.IgniteUtils.isPow2; + +import org.apache.ignite.raft.jraft.entity.LogId; + +/** + * Cyclic buffer to cache several last term values for log storage. + */ +public class TermCache { + private final int mask; + private final long[] indexes; + private final long[] terms; + + // Head position. -1 means the cache is empty. + private int head = -1; + + // Tail position. Might be equal to head if the cache only has a single term. + private int tail; + + /** + * Constructor. + * + * @param capacity Cache capacity. Must be a power of 2. Should be a small value, term update is a rare operation. + */ + public TermCache(int capacity) { + assert isPow2(capacity) : "Capacity must be a power of 2"; + + this.mask = capacity - 1; + this.indexes = new long[capacity]; + this.terms = new long[capacity]; + } + + /** + * Should be called when appending a new log entry. + */ + public void append(LogId id) { + if (isEmpty()) { + head = 0; + indexes[tail] = id.getIndex(); + terms[tail] = id.getTerm(); + + return; + } + + // Term has not changed, nothing to update. + if (terms[tail] == id.getTerm()) { + return; + } + + tail = next(tail); + indexes[tail] = id.getIndex(); + terms[tail] = id.getTerm(); + + // Handle buffer overflow by moving head to the next position. + if (tail == head) { + head = next(head); + } + } + + private int prev(int i) { + return (i - 1) & mask; + } + + private int next(int i) { + return (i + 1) & mask; + } + + private boolean isEmpty() { + return head == -1; + } + + private int findIndex(long idx) { + // Could be replaced with a binary search, but why bother for such a small cache. + for (int i = tail; i != head; i = prev(i)) { + if (idx >= indexes[i]) { + return i; + } + } + + return head; + } + + /** + * Lookup term for the given index. Returns {@code -1} if the index is not found in the cache. + */ + public long lookup(long idx) { + if (isEmpty() || idx < indexes[head]) { + return -1; + } + + return terms[findIndex(idx)]; + } + + /** + * Resets the cache to the initial state. + */ + public void reset() { + head = -1; + tail = 0; + } + + /** + * Truncates the cache to the given index, deleting all information for indexes greater than or equal to the given one. + */ + public void truncateTail(long idx) { + if (isEmpty() || idx < indexes[head]) { + reset(); + + return; + } + + tail = findIndex(idx); + + if (indexes[tail] == idx) { + if (head == tail) { + reset(); + } else { + tail = prev(tail); + } + } + } +} diff --git a/modules/raft/src/main/java/org/apache/ignite/raft/jraft/storage/impl/LogManagerImpl.java b/modules/raft/src/main/java/org/apache/ignite/raft/jraft/storage/impl/LogManagerImpl.java index 9c646954c85..c526fe94881 100644 --- a/modules/raft/src/main/java/org/apache/ignite/raft/jraft/storage/impl/LogManagerImpl.java +++ b/modules/raft/src/main/java/org/apache/ignite/raft/jraft/storage/impl/LogManagerImpl.java @@ -17,7 +17,6 @@ package org.apache.ignite.raft.jraft.storage.impl; import com.lmax.disruptor.EventHandler; -import com.lmax.disruptor.EventTranslator; import com.lmax.disruptor.RingBuffer; import java.util.ArrayList; import java.util.HashMap; @@ -30,13 +29,13 @@ import java.util.concurrent.locks.ReentrantReadWriteLock; import org.apache.ignite.internal.logger.IgniteLogger; import org.apache.ignite.internal.logger.Loggers; +import org.apache.ignite.internal.raft.storage.TermCache; import org.apache.ignite.raft.jraft.FSMCaller; import org.apache.ignite.raft.jraft.Status; import org.apache.ignite.raft.jraft.conf.Configuration; import org.apache.ignite.raft.jraft.conf.ConfigurationEntry; import org.apache.ignite.raft.jraft.conf.ConfigurationManager; import org.apache.ignite.raft.jraft.core.NodeMetrics; -import org.apache.ignite.raft.jraft.disruptor.DisruptorEventType; import org.apache.ignite.raft.jraft.disruptor.NodeIdAware; import org.apache.ignite.raft.jraft.disruptor.StripedDisruptor; import org.apache.ignite.raft.jraft.entity.EnumOutter.EntryType; @@ -59,7 +58,6 @@ import org.apache.ignite.raft.jraft.util.DisruptorMetricSet; import org.apache.ignite.raft.jraft.util.Requires; import org.apache.ignite.raft.jraft.util.SegmentList; -import org.apache.ignite.raft.jraft.util.ThreadHelper; import org.apache.ignite.raft.jraft.util.Utils; /** @@ -83,6 +81,7 @@ public class LogManagerImpl implements LogManager { private LogId diskId = new LogId(0, 0); // Last log entry written to disk. private LogId appliedId = new LogId(0, 0); private final SegmentList logsInMemory = new SegmentList<>(true); + private final TermCache termCache = new TermCache(8); private volatile long firstLogIndex; private volatile long lastLogIndex; private volatile LogId lastSnapshotId = new LogId(0, 0); @@ -321,6 +320,10 @@ public void appendEntries(final List entries, final StableClosure done if (!entries.isEmpty()) { done.setFirstLogIndex(entries.get(0).getId().getIndex()); this.logsInMemory.addAll(entries); + + for (LogEntry entry : entries) { + this.termCache.append(entry.getId()); + } } done.setEntries(entries); @@ -821,9 +824,10 @@ public long getTerm(final long index) { if (index > this.lastLogIndex || index < this.firstLogIndex) { return 0; } - final LogEntry entry = getEntryFromMemory(index); - if (entry != null) { - return entry.getId().getTerm(); + + long term = termCache.lookup(index); + if (term != -1) { + return term; } } finally { @@ -913,10 +917,12 @@ private long unsafeGetTerm(final long index) { if (index > this.lastLogIndex || index < this.firstLogIndex) { return 0; } - final LogEntry entry = getEntryFromMemory(index); - if (entry != null) { - return entry.getId().getTerm(); + + long term = termCache.lookup(index); + if (term != -1) { + return term; } + return getTermFromLogStorage(index); } @@ -1029,6 +1035,7 @@ private boolean reset(final long nextLogIndex) { this.writeLock.lock(); try { this.logsInMemory.clear(); + this.termCache.reset(); this.firstLogIndex = nextLogIndex; this.lastLogIndex = nextLogIndex - 1; this.configManager.truncatePrefix(this.firstLogIndex); @@ -1050,6 +1057,7 @@ private void unsafeTruncateSuffix(final long lastIndexKept, final Lock lock) { } this.logsInMemory.removeFromLastWhen(entry -> entry.getId().getIndex() > lastIndexKept); + termCache.truncateTail(lastIndexKept + 1); this.lastLogIndex = lastIndexKept; final long lastTermKept = unsafeGetTerm(lastIndexKept); diff --git a/modules/raft/src/test/java/org/apache/ignite/internal/raft/storage/TermCacheTest.java b/modules/raft/src/test/java/org/apache/ignite/internal/raft/storage/TermCacheTest.java new file mode 100644 index 00000000000..4e3b33308c9 --- /dev/null +++ b/modules/raft/src/test/java/org/apache/ignite/internal/raft/storage/TermCacheTest.java @@ -0,0 +1,90 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.raft.storage; + + +import static org.junit.jupiter.api.Assertions.assertEquals; + +import org.apache.ignite.raft.jraft.entity.LogId; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +class TermCacheTest { + private TermCache termCache; + + @BeforeEach + void setUp() { + termCache = new TermCache(4); + } + + @Test + void testAppendAndLookup() { + termCache.append(new LogId(1, 1)); + termCache.append(new LogId(2, 2)); + termCache.append(new LogId(3, 3)); + + assertEquals(-1, termCache.lookup(0)); + assertEquals(1, termCache.lookup(1)); + assertEquals(2, termCache.lookup(2)); + assertEquals(3, termCache.lookup(3)); + assertEquals(3, termCache.lookup(4)); + } + + @Test + void testAppendSameTerm() { + termCache.append(new LogId(1, 1)); + termCache.append(new LogId(2, 1)); + + assertEquals(1, termCache.lookup(1)); + assertEquals(1, termCache.lookup(2)); + } + + @Test + void testReset() { + termCache.append(new LogId(1, 1)); + termCache.reset(); + + assertEquals(-1, termCache.lookup(1)); + } + + @Test + void testTruncateTail() { + termCache.append(new LogId(1, 1)); + termCache.append(new LogId(2, 1)); + termCache.append(new LogId(3, 2)); + + termCache.truncateTail(2); + + assertEquals(1, termCache.lookup(1)); + assertEquals(1, termCache.lookup(2)); + assertEquals(1, termCache.lookup(3)); + } + + @Test + void testTruncateTailExactMatch() { + termCache.append(new LogId(1, 1)); + termCache.append(new LogId(2, 2)); + termCache.append(new LogId(3, 2)); + + termCache.truncateTail(2); + + assertEquals(1, termCache.lookup(1)); + assertEquals(1, termCache.lookup(2)); + assertEquals(1, termCache.lookup(3)); + } +} From e3f7c9a503550ecab2d0c2c434e54f3288b67fd8 Mon Sep 17 00:00:00 2001 From: Alexey Scherbakov Date: Fri, 10 Jan 2025 10:26:11 +0300 Subject: [PATCH 006/113] IGNITE-23712 Get rid of ticks on RW requests. --- .../storage/InternalTableImpl.java | 33 ++++++++++--------- 1 file changed, 17 insertions(+), 16 deletions(-) diff --git a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/storage/InternalTableImpl.java b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/storage/InternalTableImpl.java index be91eb7f07c..eaf214ee076 100644 --- a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/storage/InternalTableImpl.java +++ b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/storage/InternalTableImpl.java @@ -120,6 +120,7 @@ import org.apache.ignite.internal.table.distributed.storage.PartitionScanPublisher.InflightBatchRequestTracker; import org.apache.ignite.internal.tx.HybridTimestampTracker; import org.apache.ignite.internal.tx.InternalTransaction; +import org.apache.ignite.internal.tx.TransactionIds; import org.apache.ignite.internal.tx.TxManager; import org.apache.ignite.internal.tx.impl.TransactionInflights; import org.apache.ignite.internal.tx.storage.state.TxStateTableStorage; @@ -543,7 +544,7 @@ private CompletableFuture> enlistCursorInTx( (enlistmentConsistencyToken) -> TABLE_MESSAGES_FACTORY.readWriteScanRetrieveBatchReplicaRequest() .groupId(serializeTablePartitionId(partGroupId)) .tableId(tableId) - .timestamp(clockService.now()) + .timestamp(tx.startTimestamp()) .transactionId(tx.id()) .scanId(scanId) .indexToUse(indexId) @@ -903,7 +904,7 @@ public CompletableFuture get(BinaryRowEx keyRow, @Nullable InternalTr .transactionId(txo.id()) .enlistmentConsistencyToken(enlistmentConsistencyToken) .requestType(RW_GET) - .timestamp(clockService.now()) + .timestamp(txo.startTimestamp()) .full(false) .coordinatorId(txo.coordinatorId()) .build(), @@ -1058,7 +1059,7 @@ private ReadWriteMultiRowPkReplicaRequest readWriteMultiRowPkReplicaRequest( .transactionId(tx.id()) .enlistmentConsistencyToken(enlistmentConsistencyToken) .requestType(requestType) - .timestamp(clockService.now()) + .timestamp(tx.startTimestamp()) .full(full) .coordinatorId(tx.coordinatorId()) .build(); @@ -1127,7 +1128,7 @@ public CompletableFuture upsert(BinaryRowEx row, @Nullable InternalTransac .transactionId(txo.id()) .enlistmentConsistencyToken(enlistmentConsistencyToken) .requestType(RW_UPSERT) - .timestamp(txo.startTimestamp()) // TODO https://issues.apache.org/jira/browse/IGNITE-23712 + .timestamp(txo.startTimestamp()) .full(txo.implicit()) .coordinatorId(txo.coordinatorId()) .build(), @@ -1214,7 +1215,7 @@ public CompletableFuture getAndUpsert(BinaryRowEx row, InternalTransa .transactionId(txo.id()) .enlistmentConsistencyToken(enlistmentConsistencyToken) .requestType(RW_GET_AND_UPSERT) - .timestamp(clockService.now()) + .timestamp(txo.startTimestamp()) .full(txo.implicit()) .coordinatorId(txo.coordinatorId()) .build(), @@ -1237,7 +1238,7 @@ public CompletableFuture insert(BinaryRowEx row, InternalTransaction tx .transactionId(txo.id()) .enlistmentConsistencyToken(enlistmentConsistencyToken) .requestType(RW_INSERT) - .timestamp(clockService.now()) + .timestamp(txo.startTimestamp()) .full(txo.implicit()) .coordinatorId(txo.coordinatorId()) .build(), @@ -1295,7 +1296,7 @@ private ReadWriteMultiRowReplicaRequest readWriteMultiRowReplicaRequest( .transactionId(tx.id()) .enlistmentConsistencyToken(enlistmentConsistencyToken) .requestType(requestType) - .timestamp(clockService.now()) + .timestamp(tx.startTimestamp()) .full(full) .coordinatorId(tx.coordinatorId()) .build(); @@ -1316,7 +1317,7 @@ public CompletableFuture replace(BinaryRowEx row, InternalTransaction t .transactionId(txo.id()) .enlistmentConsistencyToken(enlistmentConsistencyToken) .requestType(RW_REPLACE_IF_EXIST) - .timestamp(clockService.now()) + .timestamp(txo.startTimestamp()) .full(txo.implicit()) .coordinatorId(txo.coordinatorId()) .build(), @@ -1343,7 +1344,7 @@ public CompletableFuture replace(BinaryRowEx oldRow, BinaryRowEx newRow .transactionId(txo.id()) .enlistmentConsistencyToken(enlistmentConsistencyToken) .requestType(RW_REPLACE) - .timestamp(clockService.now()) + .timestamp(txo.startTimestamp()) .full(txo.implicit()) .coordinatorId(txo.coordinatorId()) .build(), @@ -1368,7 +1369,7 @@ public CompletableFuture getAndReplace(BinaryRowEx row, InternalTrans .transactionId(txo.id()) .enlistmentConsistencyToken(enlistmentConsistencyToken) .requestType(RW_GET_AND_REPLACE) - .timestamp(clockService.now()) + .timestamp(txo.startTimestamp()) .full(txo.implicit()) .coordinatorId(txo.coordinatorId()) .build(), @@ -1391,7 +1392,7 @@ public CompletableFuture delete(BinaryRowEx keyRow, InternalTransaction .transactionId(txo.id()) .enlistmentConsistencyToken(enlistmentConsistencyToken) .requestType(RW_DELETE) - .timestamp(clockService.now()) + .timestamp(txo.startTimestamp()) .full(txo.implicit()) .coordinatorId(txo.coordinatorId()) .build(), @@ -1414,7 +1415,7 @@ public CompletableFuture deleteExact(BinaryRowEx oldRow, InternalTransa .transactionId(txo.id()) .enlistmentConsistencyToken(enlistmentConsistencyToken) .requestType(RW_DELETE_EXACT) - .timestamp(clockService.now()) + .timestamp(txo.startTimestamp()) .full(txo.implicit()) .coordinatorId(txo.coordinatorId()) .build(), @@ -1439,7 +1440,7 @@ public CompletableFuture getAndDelete(BinaryRowEx row, InternalTransa .transactionId(txo.id()) .enlistmentConsistencyToken(enlistmentConsistencyToken) .requestType(RW_GET_AND_DELETE) - .timestamp(clockService.now()) + .timestamp(txo.startTimestamp()) .full(txo.implicit()) .coordinatorId(txo.coordinatorId()) .build(), @@ -1750,7 +1751,7 @@ protected CompletableFuture> retrieveBatch(long scanId, in ReadWriteScanRetrieveBatchReplicaRequest request = TABLE_MESSAGES_FACTORY.readWriteScanRetrieveBatchReplicaRequest() .groupId(serializeTablePartitionId(tablePartitionId)) .tableId(tableId) - .timestamp(clockService.now()) + .timestamp(TransactionIds.beginTimestamp(txId)) .transactionId(txId) .scanId(scanId) .indexToUse(indexId) @@ -1997,7 +1998,7 @@ protected CompletableFuture> enlist(int partId, @Override public CompletableFuture partitionLocation(TablePartitionId tablePartitionId) { - return partitionMeta(tablePartitionId, clockService.now()).thenApply(this::getClusterNode); + return partitionMeta(tablePartitionId, clockService.current()).thenApply(this::getClusterNode); } private CompletableFuture partitionMeta(TablePartitionId tablePartitionId, HybridTimestamp at) { @@ -2119,7 +2120,7 @@ public ScheduledExecutorService streamerFlushExecutor() { @Override public CompletableFuture estimatedSize() { - HybridTimestamp now = clockService.now(); + HybridTimestamp now = clockService.current(); var invokeFutures = new CompletableFuture[partitions]; From d29efc5b3d33f020d564529947791be75b8ffdbf Mon Sep 17 00:00:00 2001 From: Aleksandr Pakhomov Date: Fri, 10 Jan 2025 14:14:53 +0300 Subject: [PATCH 007/113] IGNITE-24163 Optimize EventLog (#5004) * Get rid of unnecessary locking in `log` method * Add one more "lazy" log method to the interface * Return null instead of empty collection (isEmpty() call was contributing about 20% execution time to EventLog.log method) --- .../ignite/client/handler/TestServer.java | 15 +++- .../org/apache/ignite/client/TestServer.java | 15 +++- .../impl/ItEventLogConfigurationTest.java | 16 ++--- .../internal/eventlog/api/EventFactory.java | 3 +- .../internal/eventlog/api/EventLog.java | 13 +++- .../internal/eventlog/api/IgniteEvents.java | 20 +++--- .../eventlog/impl/ChannelRegistry.java | 8 ++- .../ConfigurationBasedChannelRegistry.java | 65 +++++++---------- .../impl/ConfigurationBasedSinkRegistry.java | 56 ++++++--------- .../eventlog/impl/EventChannelImpl.java | 8 ++- .../internal/eventlog/impl/EventLogImpl.java | 22 +++++- .../internal/eventlog/impl/SinkFactory.java | 2 +- .../internal/eventlog/impl/SinkRegistry.java | 22 +++++- ...ConfigurationBasedChannelRegistryTest.java | 2 +- .../ConfigurationBasedSinkRegistryTest.java | 2 +- .../internal/eventlog/impl/EventLogTest.java | 71 ++++++++++++++++--- .../eventlog/impl/WebhookSinkTest.java | 18 ++--- .../internal/rest/RestComponentTest.java | 14 +++- .../AuthenticationManagerImpl.java | 13 ++-- .../AuthenticationManagerImplTest.java | 15 +++- 20 files changed, 267 insertions(+), 133 deletions(-) diff --git a/modules/client-handler/src/integrationTest/java/org/apache/ignite/client/handler/TestServer.java b/modules/client-handler/src/integrationTest/java/org/apache/ignite/client/handler/TestServer.java index 4c20bef046a..344addd3886 100644 --- a/modules/client-handler/src/integrationTest/java/org/apache/ignite/client/handler/TestServer.java +++ b/modules/client-handler/src/integrationTest/java/org/apache/ignite/client/handler/TestServer.java @@ -24,11 +24,14 @@ import java.util.List; import java.util.UUID; +import java.util.function.Supplier; import org.apache.ignite.client.handler.configuration.ClientConnectorConfiguration; import org.apache.ignite.internal.catalog.CatalogService; import org.apache.ignite.internal.cluster.management.ClusterTag; import org.apache.ignite.internal.cluster.management.network.messages.CmgMessagesFactory; import org.apache.ignite.internal.compute.IgniteComputeInternal; +import org.apache.ignite.internal.eventlog.api.Event; +import org.apache.ignite.internal.eventlog.api.EventLog; import org.apache.ignite.internal.hlc.HybridClockImpl; import org.apache.ignite.internal.hlc.TestClockService; import org.apache.ignite.internal.lowwatermark.TestLowWatermark; @@ -80,7 +83,17 @@ public TestServer(ClientConnectorConfiguration clientConnectorConfiguration, Net this.testSslConfig = testSslConfig; this.authenticationManager = securityConfiguration == null ? new DummyAuthenticationManager() - : new AuthenticationManagerImpl(securityConfiguration, ign -> {}); + : new AuthenticationManagerImpl(securityConfiguration, new EventLog() { + @Override + public void log(Event event) { + + } + + @Override + public void log(String type, Supplier eventProvider) { + + } + }); this.clientConnectorConfiguration = clientConnectorConfiguration; this.networkConfiguration = networkConfiguration; diff --git a/modules/client/src/test/java/org/apache/ignite/client/TestServer.java b/modules/client/src/test/java/org/apache/ignite/client/TestServer.java index 9e662132c0b..3f6a14256dd 100644 --- a/modules/client/src/test/java/org/apache/ignite/client/TestServer.java +++ b/modules/client/src/test/java/org/apache/ignite/client/TestServer.java @@ -36,6 +36,7 @@ import java.util.Objects; import java.util.UUID; import java.util.function.Function; +import java.util.function.Supplier; import org.apache.ignite.Ignite; import org.apache.ignite.client.fakes.FakeIgnite; import org.apache.ignite.client.fakes.FakeInternalTable; @@ -57,6 +58,8 @@ import org.apache.ignite.internal.configuration.NodeConfiguration; import org.apache.ignite.internal.configuration.storage.TestConfigurationStorage; import org.apache.ignite.internal.configuration.validation.TestConfigurationValidator; +import org.apache.ignite.internal.eventlog.api.Event; +import org.apache.ignite.internal.eventlog.api.EventLog; import org.apache.ignite.internal.hlc.HybridClock; import org.apache.ignite.internal.hlc.HybridClockImpl; import org.apache.ignite.internal.hlc.TestClockService; @@ -221,7 +224,17 @@ public TestServer( if (securityConfiguration == null) { authenticationManager = new DummyAuthenticationManager(); } else { - authenticationManager = new AuthenticationManagerImpl(securityConfiguration, ign -> {}); + authenticationManager = new AuthenticationManagerImpl(securityConfiguration, new EventLog() { + @Override + public void log(Event event) { + + } + + @Override + public void log(String type, Supplier eventProvider) { + + } + }); assertThat(authenticationManager.startAsync(componentContext), willCompleteSuccessfully()); } diff --git a/modules/eventlog/src/integrationTest/java/org/apache/ignite/internal/eventlog/impl/ItEventLogConfigurationTest.java b/modules/eventlog/src/integrationTest/java/org/apache/ignite/internal/eventlog/impl/ItEventLogConfigurationTest.java index 2f481c34402..484b31fcd5e 100644 --- a/modules/eventlog/src/integrationTest/java/org/apache/ignite/internal/eventlog/impl/ItEventLogConfigurationTest.java +++ b/modules/eventlog/src/integrationTest/java/org/apache/ignite/internal/eventlog/impl/ItEventLogConfigurationTest.java @@ -81,7 +81,7 @@ void configureChannelAndTestSink() throws Exception { .user(EventUser.system()) .build(); - eventLog.log(() -> event); + eventLog.log(event); // Then event is written into the sink. assertThat(inMemoryCollectionSink.events(), contains(event)); @@ -110,7 +110,7 @@ void configureTestSinkAndChannel() throws Exception { .user(EventUser.system()) .build(); - eventLog.log(() -> event); + eventLog.log(event); // Then event is written into the sink. assertThat(inMemoryCollectionSink.events(), contains(event)); @@ -139,7 +139,7 @@ void channelConfigurationChanges() throws Exception { .user(EventUser.system()) .build(); - eventLog.log(() -> event); + eventLog.log(event); // Then event is written into the sink. assertThat(inMemoryCollectionSink.events(), hasSize(1)); @@ -150,7 +150,7 @@ void channelConfigurationChanges() throws Exception { .user(EventUser.system()) .build(); - eventLog.log(() -> event2); + eventLog.log(event2); // Then the event is not written into the sink. assertThat(inMemoryCollectionSink.events(), hasSize(1)); @@ -168,7 +168,7 @@ void channelConfigurationChanges() throws Exception { assertThat(inMemoryCollectionSink.events(), hasSize(1)); // When log event2 again. - eventLog.log(() -> event2); + eventLog.log(event2); // Then the event2 is written into the sink. assertThat(inMemoryCollectionSink.events(), hasSize(2)); @@ -197,7 +197,7 @@ void enableDisableChannel() throws Exception { .user(EventUser.system()) .build(); - eventLog.log(() -> event); + eventLog.log(event); // Then event is written into the sink. assertThat(inMemoryCollectionSink.events(), hasSize(1)); @@ -209,7 +209,7 @@ void enableDisableChannel() throws Exception { )).get(); // When log event again. - eventLog.log(() -> event); + eventLog.log(event); // Then the event is not written into the sink. assertThat(inMemoryCollectionSink.events(), hasSize(1)); @@ -221,7 +221,7 @@ void enableDisableChannel() throws Exception { )).get(); // When log event again. - eventLog.log(() -> event); + eventLog.log(event); // Then the event is written into the sink. assertThat(inMemoryCollectionSink.events(), hasSize(2)); diff --git a/modules/eventlog/src/main/java/org/apache/ignite/internal/eventlog/api/EventFactory.java b/modules/eventlog/src/main/java/org/apache/ignite/internal/eventlog/api/EventFactory.java index d652ead4162..7e65d2dc595 100644 --- a/modules/eventlog/src/main/java/org/apache/ignite/internal/eventlog/api/EventFactory.java +++ b/modules/eventlog/src/main/java/org/apache/ignite/internal/eventlog/api/EventFactory.java @@ -35,7 +35,8 @@ public interface EventFactory { /** * Creates an event builder with the event type defined. The type is set by the factory. For example, - * {@link IgniteEvents.CONNECTION_CLOSED.build} will return a builder with {@link IgniteEventType.CONNECTION_CLOSED} type set. + * {@link IgniteEvents.USER_AUTHENTICATION_SUCCESS.builder} will return a builder with + * {@link IgniteEventType.USER_AUTHENTICATION_SUCCESS} type set. */ EventBuilder builder(); } diff --git a/modules/eventlog/src/main/java/org/apache/ignite/internal/eventlog/api/EventLog.java b/modules/eventlog/src/main/java/org/apache/ignite/internal/eventlog/api/EventLog.java index 922d3754e15..bd864dea9fc 100644 --- a/modules/eventlog/src/main/java/org/apache/ignite/internal/eventlog/api/EventLog.java +++ b/modules/eventlog/src/main/java/org/apache/ignite/internal/eventlog/api/EventLog.java @@ -44,7 +44,16 @@ public interface EventLog { /** * Writes event into every channel this event relates to. * - * @param eventProvider Event provider. + * @param event The event to log. */ - void log(Supplier eventProvider); + void log(Event event); + + /** + * Lazy version of {@link #log(Event)}. It first checks if the events of the given type are enabled and then logs the event. INVARIANT: + * type must be a valid event type and eventProvider must provide an event of the same type. + * + * @param type the type of the event. + * @param eventProvider the event provider. + */ + void log(String type, Supplier eventProvider); } diff --git a/modules/eventlog/src/main/java/org/apache/ignite/internal/eventlog/api/IgniteEvents.java b/modules/eventlog/src/main/java/org/apache/ignite/internal/eventlog/api/IgniteEvents.java index 1b7aae4312f..41d007bcd97 100644 --- a/modules/eventlog/src/main/java/org/apache/ignite/internal/eventlog/api/IgniteEvents.java +++ b/modules/eventlog/src/main/java/org/apache/ignite/internal/eventlog/api/IgniteEvents.java @@ -29,22 +29,26 @@ *
{@code IgniteEvents.USER_AUTHENTICATION_SUCCESS.create(EventUser.system());}
*/ public final class IgniteEvents implements EventFactory { - public static final IgniteEvents USER_AUTHENTICATION_SUCCESS = new IgniteEvents(IgniteEventType.USER_AUTHENTICATION_SUCCESS.name()); - public static final IgniteEvents USER_AUTHENTICATION_FAILURE = new IgniteEvents(IgniteEventType.USER_AUTHENTICATION_FAILURE.name()); + public static final IgniteEvents USER_AUTHENTICATION_SUCCESS = new IgniteEvents(IgniteEventType.USER_AUTHENTICATION_SUCCESS); + public static final IgniteEvents USER_AUTHENTICATION_FAILURE = new IgniteEvents(IgniteEventType.USER_AUTHENTICATION_FAILURE); - public static final IgniteEvents CLIENT_CONNECTION_ESTABLISHED = new IgniteEvents(IgniteEventType.CLIENT_CONNECTION_ESTABLISHED.name()); - public static final IgniteEvents CLIENT_CONNECTION_CLOSED = new IgniteEvents(IgniteEventType.CLIENT_CONNECTION_CLOSED.name()); + public static final IgniteEvents CLIENT_CONNECTION_ESTABLISHED = new IgniteEvents(IgniteEventType.CLIENT_CONNECTION_ESTABLISHED); + public static final IgniteEvents CLIENT_CONNECTION_CLOSED = new IgniteEvents(IgniteEventType.CLIENT_CONNECTION_CLOSED); - private final String type; + private final IgniteEventType type; - private IgniteEvents(String type) { + private IgniteEvents(IgniteEventType type) { this.type = type; } + public String type() { + return type.name(); + } + @Override public Event create(EventUser user) { return Event.builder() - .type(type) + .type(type.name()) .user(user) .timestamp(System.currentTimeMillis()) .build(); @@ -52,6 +56,6 @@ public Event create(EventUser user) { @Override public EventBuilder builder() { - return new EventBuilder().type(type); + return new EventBuilder().type(type.name()); } } diff --git a/modules/eventlog/src/main/java/org/apache/ignite/internal/eventlog/impl/ChannelRegistry.java b/modules/eventlog/src/main/java/org/apache/ignite/internal/eventlog/impl/ChannelRegistry.java index 74b7415dfe4..1728b55b90a 100644 --- a/modules/eventlog/src/main/java/org/apache/ignite/internal/eventlog/impl/ChannelRegistry.java +++ b/modules/eventlog/src/main/java/org/apache/ignite/internal/eventlog/impl/ChannelRegistry.java @@ -19,6 +19,7 @@ import java.util.Set; import org.apache.ignite.internal.eventlog.api.EventChannel; +import org.jetbrains.annotations.Nullable; /** * Channel registry. The only way to send an event into channel is to get the channel from this registry. @@ -36,8 +37,9 @@ interface ChannelRegistry { /** * Get all channels that can handle the given event type. * - * @param igniteEventType Ignite event type. - * @return Set of channels. + * @param type Ignite event type. + * @return Set of channels or {@code null} if there are no channels for the given event type. */ - Set findAllChannelsByEventType(String igniteEventType); + @Nullable + Set findAllChannelsByEventType(String type); } diff --git a/modules/eventlog/src/main/java/org/apache/ignite/internal/eventlog/impl/ConfigurationBasedChannelRegistry.java b/modules/eventlog/src/main/java/org/apache/ignite/internal/eventlog/impl/ConfigurationBasedChannelRegistry.java index 272e8a2f42a..5f036a7a6b6 100644 --- a/modules/eventlog/src/main/java/org/apache/ignite/internal/eventlog/impl/ConfigurationBasedChannelRegistry.java +++ b/modules/eventlog/src/main/java/org/apache/ignite/internal/eventlog/impl/ConfigurationBasedChannelRegistry.java @@ -25,8 +25,6 @@ import java.util.Map; import java.util.Set; import java.util.concurrent.CompletableFuture; -import java.util.concurrent.locks.ReadWriteLock; -import java.util.concurrent.locks.ReentrantReadWriteLock; import java.util.stream.Collectors; import org.apache.ignite.configuration.NamedListView; import org.apache.ignite.configuration.notifications.ConfigurationListener; @@ -34,18 +32,16 @@ import org.apache.ignite.internal.eventlog.api.EventChannel; import org.apache.ignite.internal.eventlog.config.schema.ChannelView; import org.apache.ignite.internal.eventlog.config.schema.EventLogConfiguration; +import org.jetbrains.annotations.Nullable; class ConfigurationBasedChannelRegistry implements ChannelRegistry { - private final ReadWriteLock guard; + private volatile Map cache; - private final Map cache; - - private final Map> typeCache; + private volatile Map> typeCache; private final SinkRegistry sinkRegistry; ConfigurationBasedChannelRegistry(EventLogConfiguration cfg, SinkRegistry sinkRegistry) { - this.guard = new ReentrantReadWriteLock(); this.cache = new HashMap<>(); this.typeCache = new HashMap<>(); this.sinkRegistry = sinkRegistry; @@ -55,23 +51,13 @@ class ConfigurationBasedChannelRegistry implements ChannelRegistry { @Override public EventChannel getByName(String name) { - guard.readLock().lock(); - try { - return cache.get(name); - } finally { - guard.readLock().unlock(); - } + return cache.get(name); } @Override + @Nullable public Set findAllChannelsByEventType(String igniteEventType) { - guard.readLock().lock(); - try { - Set result = typeCache.get(igniteEventType); - return result == null ? Set.of() : new HashSet<>(result); - } finally { - guard.readLock().unlock(); - } + return typeCache.get(igniteEventType); } private class CacheUpdater implements ConfigurationListener> { @@ -79,29 +65,26 @@ private class CacheUpdater implements ConfigurationListener onUpdate(ConfigurationNotificationEvent> ctx) { NamedListView newListValue = ctx.newValue(); - guard.writeLock().lock(); - - try { - cache.clear(); - typeCache.clear(); - - newListValue.forEach(view -> { - if (view.enabled()) { - EventChannel channel = createChannel(view); - cache.put(view.name(), channel); - for (String eventType : view.events()) { - typeCache.computeIfAbsent( - eventType.trim(), - t -> new HashSet<>() - ).add(channel); - } + Map newCache = new HashMap<>(); + Map> newTypeCache = new HashMap<>(); + + newListValue.forEach(view -> { + if (view.enabled()) { + EventChannel channel = createChannel(view); + newCache.put(view.name(), channel); + for (String eventType : view.events()) { + newTypeCache.computeIfAbsent( + eventType.trim(), + t -> new HashSet<>() + ).add(channel); } - }); + } + }); + + cache = newCache; + typeCache = newTypeCache; - return completedFuture(null); - } finally { - guard.writeLock().unlock(); - } + return completedFuture(null); } private EventChannel createChannel(ChannelView view) { diff --git a/modules/eventlog/src/main/java/org/apache/ignite/internal/eventlog/impl/ConfigurationBasedSinkRegistry.java b/modules/eventlog/src/main/java/org/apache/ignite/internal/eventlog/impl/ConfigurationBasedSinkRegistry.java index ef0830348fc..8364eb844b4 100644 --- a/modules/eventlog/src/main/java/org/apache/ignite/internal/eventlog/impl/ConfigurationBasedSinkRegistry.java +++ b/modules/eventlog/src/main/java/org/apache/ignite/internal/eventlog/impl/ConfigurationBasedSinkRegistry.java @@ -24,8 +24,6 @@ import java.util.Map; import java.util.Set; import java.util.concurrent.CompletableFuture; -import java.util.concurrent.locks.ReadWriteLock; -import java.util.concurrent.locks.ReentrantReadWriteLock; import org.apache.ignite.configuration.NamedListView; import org.apache.ignite.configuration.notifications.ConfigurationListener; import org.apache.ignite.configuration.notifications.ConfigurationNotificationEvent; @@ -34,16 +32,13 @@ import org.apache.ignite.internal.eventlog.config.schema.SinkView; class ConfigurationBasedSinkRegistry implements SinkRegistry { - private final ReadWriteLock guard; + private volatile Map> cache; - private final Map cache; - - private final Map> cacheByChannel; + private volatile Map>> cacheByChannel; private final SinkFactory sinkFactory; ConfigurationBasedSinkRegistry(EventLogConfiguration cfg, SinkFactory sinkFactory) { - this.guard = new ReentrantReadWriteLock(); this.cache = new HashMap<>(); this.cacheByChannel = new HashMap<>(); this.sinkFactory = sinkFactory; @@ -52,24 +47,13 @@ class ConfigurationBasedSinkRegistry implements SinkRegistry { } @Override - public Sink getByName(String name) { - guard.readLock().lock(); - try { - return cache.get(name); - } finally { - guard.readLock().unlock(); - } + public Sink getByName(String name) { + return cache.get(name); } @Override - public Set findAllByChannel(String channel) { - guard.readLock().lock(); - try { - Set sinks = cacheByChannel.get(channel); - return sinks == null ? Set.of() : new HashSet<>(sinks); - } finally { - guard.readLock().unlock(); - } + public Set> findAllByChannel(String channel) { + return cacheByChannel.get(channel); } private class CacheUpdater implements ConfigurationListener> { @@ -77,21 +61,25 @@ private class CacheUpdater implements ConfigurationListener onUpdate(ConfigurationNotificationEvent> ctx) { NamedListView newListValue = ctx.newValue(); - guard.writeLock().lock(); - try { - cache.values().forEach(Sink::stop); - cache.clear(); + Map> newCache = new HashMap<>(); + Map>> newCacheByChannel = new HashMap<>(); - cacheByChannel.clear(); - for (SinkView sinkView : newListValue) { - Sink sink = sinkFactory.createSink(sinkView); - cache.put(sinkView.name(), sink); - cacheByChannel.computeIfAbsent(sinkView.channel(), k -> new HashSet<>()).add(sink); + for (SinkView sinkView : newListValue) { + Sink sink = sinkFactory.createSink(sinkView); + newCache.put(sinkView.name(), sink); + newCacheByChannel.computeIfAbsent(sinkView.channel(), k -> new HashSet<>()).add(sink); + } + + for (String type : cache.keySet()) { + if (!newCache.containsKey(type)) { + cache.get(type).stop(); } - return completedFuture(null); - } finally { - guard.writeLock().unlock(); } + + cache = newCache; + cacheByChannel = newCacheByChannel; + + return completedFuture(null); } } } diff --git a/modules/eventlog/src/main/java/org/apache/ignite/internal/eventlog/impl/EventChannelImpl.java b/modules/eventlog/src/main/java/org/apache/ignite/internal/eventlog/impl/EventChannelImpl.java index 28148720959..498ed1fb347 100644 --- a/modules/eventlog/src/main/java/org/apache/ignite/internal/eventlog/impl/EventChannelImpl.java +++ b/modules/eventlog/src/main/java/org/apache/ignite/internal/eventlog/impl/EventChannelImpl.java @@ -21,6 +21,7 @@ import java.util.Set; import org.apache.ignite.internal.eventlog.api.Event; import org.apache.ignite.internal.eventlog.api.EventChannel; +import org.apache.ignite.internal.eventlog.api.Sink; class EventChannelImpl implements EventChannel { private final SinkRegistry sinkRegistry; @@ -42,6 +43,11 @@ public Set types() { @Override public void log(Event event) { - sinkRegistry.findAllByChannel(channelName).forEach(s -> s.write(event)); + Set> sinks = sinkRegistry.findAllByChannel(channelName); + if (sinks == null) { + return; + } + + sinks.forEach(s -> s.write(event)); } } diff --git a/modules/eventlog/src/main/java/org/apache/ignite/internal/eventlog/impl/EventLogImpl.java b/modules/eventlog/src/main/java/org/apache/ignite/internal/eventlog/impl/EventLogImpl.java index 84ee9fdc657..01fcb79303f 100644 --- a/modules/eventlog/src/main/java/org/apache/ignite/internal/eventlog/impl/EventLogImpl.java +++ b/modules/eventlog/src/main/java/org/apache/ignite/internal/eventlog/impl/EventLogImpl.java @@ -55,9 +55,27 @@ public EventLogImpl(EventLogConfiguration cfg, Supplier clusterIdSupplier, } @Override - public void log(Supplier eventProvider) { - Event event = eventProvider.get(); + public void log(Event event) { Set channel = channelRegistry.findAllChannelsByEventType(event.getType()); + if (channel == null) { + return; + } + channel.forEach(c -> c.log(event)); } + + @Override + public void log(String type, Supplier eventProvider) { + Set channels = channelRegistry.findAllChannelsByEventType(type); + if (channels == null) { + return; + } + + Event event = eventProvider.get(); + if (!event.getType().equals(type)) { + throw new IllegalArgumentException("Event type mismatch: " + event.getType() + " != " + type); + } + + channels.forEach(c -> c.log(event)); + } } diff --git a/modules/eventlog/src/main/java/org/apache/ignite/internal/eventlog/impl/SinkFactory.java b/modules/eventlog/src/main/java/org/apache/ignite/internal/eventlog/impl/SinkFactory.java index e967122956b..ce64bf45741 100644 --- a/modules/eventlog/src/main/java/org/apache/ignite/internal/eventlog/impl/SinkFactory.java +++ b/modules/eventlog/src/main/java/org/apache/ignite/internal/eventlog/impl/SinkFactory.java @@ -25,7 +25,7 @@ /** * Factory for creating sink instances. */ -public interface SinkFactory { +interface SinkFactory { /** * Creates a sink instance. * diff --git a/modules/eventlog/src/main/java/org/apache/ignite/internal/eventlog/impl/SinkRegistry.java b/modules/eventlog/src/main/java/org/apache/ignite/internal/eventlog/impl/SinkRegistry.java index 5151ae7b913..6c64f29d40a 100644 --- a/modules/eventlog/src/main/java/org/apache/ignite/internal/eventlog/impl/SinkRegistry.java +++ b/modules/eventlog/src/main/java/org/apache/ignite/internal/eventlog/impl/SinkRegistry.java @@ -19,9 +19,27 @@ import java.util.Set; import org.apache.ignite.internal.eventlog.api.Sink; +import org.jetbrains.annotations.Nullable; +/** + * Sink registry. Follows the same principles as {@link ChannelRegistry}. + */ interface SinkRegistry { - Sink getByName(String name); + /** + * Get sink by name. + * + * @param name Sink name. + * @return Sink instance. + */ + @Nullable + Sink getByName(String name); - Set findAllByChannel(String channel); + /** + * Get all sinks that can handle the given channel. + * + * @param channel Channel name. + * @return Set of sinks or {@code null} if there are no sinks for the given channel. + */ + @Nullable + Set> findAllByChannel(String channel); } diff --git a/modules/eventlog/src/test/java/org/apache/ignite/internal/eventlog/impl/ConfigurationBasedChannelRegistryTest.java b/modules/eventlog/src/test/java/org/apache/ignite/internal/eventlog/impl/ConfigurationBasedChannelRegistryTest.java index 711725a2ade..a8458941fbb 100644 --- a/modules/eventlog/src/test/java/org/apache/ignite/internal/eventlog/impl/ConfigurationBasedChannelRegistryTest.java +++ b/modules/eventlog/src/test/java/org/apache/ignite/internal/eventlog/impl/ConfigurationBasedChannelRegistryTest.java @@ -119,7 +119,7 @@ void findAllChannelsByEventType() throws Exception { // Then registry returns the channel by type. assertThat(registry.findAllChannelsByEventType(IgniteEventType.USER_AUTHENTICATION_SUCCESS.name()), hasSize(1)); // But for another type it returns empty set. - assertThat(registry.findAllChannelsByEventType(IgniteEventType.CLIENT_CONNECTION_CLOSED.name()), hasSize(0)); + assertThat(registry.findAllChannelsByEventType(IgniteEventType.CLIENT_CONNECTION_CLOSED.name()), nullValue()); // When update configuration entry. cfg.channels().change(c -> c.update(TEST_CHANNEL, s -> { diff --git a/modules/eventlog/src/test/java/org/apache/ignite/internal/eventlog/impl/ConfigurationBasedSinkRegistryTest.java b/modules/eventlog/src/test/java/org/apache/ignite/internal/eventlog/impl/ConfigurationBasedSinkRegistryTest.java index 0216c3a9f00..445f93ccc8a 100644 --- a/modules/eventlog/src/test/java/org/apache/ignite/internal/eventlog/impl/ConfigurationBasedSinkRegistryTest.java +++ b/modules/eventlog/src/test/java/org/apache/ignite/internal/eventlog/impl/ConfigurationBasedSinkRegistryTest.java @@ -113,7 +113,7 @@ void updateConfigurationEntry() throws Exception { ); // Then then the sink can not be found by previous channel. - assertThat(registry.findAllByChannel("some"), hasSize(0)); + assertThat(registry.findAllByChannel("some"), nullValue()); // And the sink can be found by new channel. assertThat(registry.findAllByChannel(TEST_CHANNEL), hasSize(1)); diff --git a/modules/eventlog/src/test/java/org/apache/ignite/internal/eventlog/impl/EventLogTest.java b/modules/eventlog/src/test/java/org/apache/ignite/internal/eventlog/impl/EventLogTest.java index 888aa9a007e..fad37dcfa63 100644 --- a/modules/eventlog/src/test/java/org/apache/ignite/internal/eventlog/impl/EventLogTest.java +++ b/modules/eventlog/src/test/java/org/apache/ignite/internal/eventlog/impl/EventLogTest.java @@ -21,6 +21,7 @@ import static org.hamcrest.CoreMatchers.not; import static org.hamcrest.MatcherAssert.assertThat; import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; +import static org.junit.jupiter.api.Assertions.assertThrows; import java.util.ArrayList; import java.util.HashMap; @@ -62,7 +63,7 @@ void logsEventCorrectly() { // Given no channels and sinks. // Then nothing thrown. - assertDoesNotThrow(() -> eventLog.log(() -> TEST_EVENT)); + assertDoesNotThrow(() -> eventLog.log(TEST_EVENT)); // When add a channel but there is no sink. channelRegistry.register(TEST_CHANNEL_NAME, () -> channelFactory.createChannel( @@ -70,14 +71,14 @@ void logsEventCorrectly() { ); // Then nothing thrown. - assertDoesNotThrow(() -> eventLog.log(() -> TEST_EVENT)); + assertDoesNotThrow(() -> eventLog.log(TEST_EVENT)); // When add a sink for the channel. List container = new ArrayList<>(); sinkRegistry.register(TEST_CHANNEL_NAME, container::add); // And log event. - eventLog.log(() -> TEST_EVENT); + eventLog.log(TEST_EVENT); // Then event is logged. assertThat(container, hasItem(TEST_EVENT)); @@ -86,11 +87,65 @@ void logsEventCorrectly() { Event event = IgniteEvents.CLIENT_CONNECTION_CLOSED.create(TEST_USER); // Then nothing thrown. - assertDoesNotThrow(() -> eventLog.log(() -> event)); + assertDoesNotThrow(() -> eventLog.log(event)); // And the event is not logged. assertThat(container, not(hasItem(event))); } + @Test + void lazyLog() { + // Given no channels and sinks. + + // Then nothing thrown. + assertDoesNotThrow(() -> eventLog.log(TEST_EVENT.getType(), () -> TEST_EVENT)); + + // When add a channel but there is no sink. + channelRegistry.register( + TEST_CHANNEL_NAME, + () -> channelFactory.createChannel(TEST_CHANNEL_NAME, Set.of(TEST_EVENT.getType())) + ); + + // Then nothing thrown. + assertDoesNotThrow(() -> eventLog.log(TEST_EVENT.getType(), () -> TEST_EVENT)); + + // When add a sink for the channel. + List container = new ArrayList<>(); + sinkRegistry.register(TEST_CHANNEL_NAME, container::add); + + // And log event. + eventLog.log(TEST_EVENT.getType(), () -> TEST_EVENT); + + // Then event is logged. + assertThat(container, hasItem(TEST_EVENT)); + + // When log event with a type that is not supported by the channel. + Event event = IgniteEvents.CLIENT_CONNECTION_CLOSED.create(TEST_USER); + + // Then nothing thrown. + assertDoesNotThrow(() -> eventLog.log(event.getType(), () -> event)); + // And the event is not logged. + assertThat(container, not(hasItem(event))); + + } + + @Test + void lazyLogInvariant() { + // Given channel is added to the registry. + channelRegistry.register( + TEST_CHANNEL_NAME, + () -> channelFactory.createChannel(TEST_CHANNEL_NAME, Set.of(TEST_EVENT.getType())) + ); + + // Expect exception if the type does not match the provided event. + assertThrows( + IllegalArgumentException.class, + () -> eventLog.log( + TEST_EVENT.getType(), + () -> IgniteEvents.CLIENT_CONNECTION_CLOSED.create(TEST_USER) + ) + ); + } + private static class TestChannelRegistry implements ChannelRegistry { private final Map> channels; @@ -117,23 +172,23 @@ public Set findAllChannelsByEventType(String igniteEventType) { } private static class TestSinkRegistry implements SinkRegistry { - private final Map sinks; + private final Map> sinks; private TestSinkRegistry() { sinks = new HashMap<>(); } - void register(String name, Sink sink) { + void register(String name, Sink sink) { sinks.put(name, sink); } @Override - public Sink getByName(String name) { + public Sink getByName(String name) { return sinks.get(name); } @Override - public Set findAllByChannel(String channel) { + public Set> findAllByChannel(String channel) { if (!sinks.containsKey(channel)) { return Set.of(); } diff --git a/modules/eventlog/src/test/java/org/apache/ignite/internal/eventlog/impl/WebhookSinkTest.java b/modules/eventlog/src/test/java/org/apache/ignite/internal/eventlog/impl/WebhookSinkTest.java index a4ae893511b..f9fb698c374 100644 --- a/modules/eventlog/src/test/java/org/apache/ignite/internal/eventlog/impl/WebhookSinkTest.java +++ b/modules/eventlog/src/test/java/org/apache/ignite/internal/eventlog/impl/WebhookSinkTest.java @@ -19,7 +19,7 @@ import static org.apache.ignite.internal.testframework.matchers.CompletableFutureMatcher.willCompleteSuccessfully; import static org.hamcrest.MatcherAssert.assertThat; -import static org.hamcrest.Matchers.containsInAnyOrder; +import static org.hamcrest.Matchers.hasItems; import static org.hamcrest.Matchers.hasSize; import static org.mockserver.matchers.MatchType.ONLY_MATCHING_FIELDS; import static org.mockserver.model.HttpRequest.request; @@ -165,20 +165,16 @@ void shouldSkipSendingEvents() { @Test void shouldRemoveEventsIfQueueIsFull() { - WebhookSink sink = createSink(c -> c.changeQueueSize(2)); + var user1Event = IgniteEvents.USER_AUTHENTICATION_SUCCESS.create(EventUser.of("user1", "basicProvider")); + var user2Event = IgniteEvents.USER_AUTHENTICATION_FAILURE.create(EventUser.of("user2", "basicProvider")); + var user3Event = IgniteEvents.CLIENT_CONNECTION_ESTABLISHED.create(EventUser.of("user3", "basicProvider")); - Stream events = Stream.of( - IgniteEvents.USER_AUTHENTICATION_SUCCESS.create(EventUser.of("user1", "basicProvider")), - IgniteEvents.USER_AUTHENTICATION_FAILURE.create(EventUser.of("user2", "basicProvider")), - IgniteEvents.CLIENT_CONNECTION_ESTABLISHED.create(EventUser.of("user3", "basicProvider")) - ); + Stream events = Stream.of(user1Event, user2Event, user3Event); + WebhookSink sink = createSink(c -> c.changeQueueSize(2)); events.forEach(sink::write); - assertThat(sink.getEvents(), containsInAnyOrder( - IgniteEvents.USER_AUTHENTICATION_FAILURE.create(EventUser.of("user2", "basicProvider")), - IgniteEvents.CLIENT_CONNECTION_ESTABLISHED.create(EventUser.of("user3", "basicProvider")) - )); + assertThat(sink.getEvents(), hasItems(user2Event, user3Event)); } @ParameterizedTest diff --git a/modules/rest/src/test/java/org/apache/ignite/internal/rest/RestComponentTest.java b/modules/rest/src/test/java/org/apache/ignite/internal/rest/RestComponentTest.java index dc32cebdfac..75982bc7ab9 100644 --- a/modules/rest/src/test/java/org/apache/ignite/internal/rest/RestComponentTest.java +++ b/modules/rest/src/test/java/org/apache/ignite/internal/rest/RestComponentTest.java @@ -43,6 +43,8 @@ import org.apache.ignite.internal.configuration.storage.TestConfigurationStorage; import org.apache.ignite.internal.configuration.testframework.InjectConfiguration; import org.apache.ignite.internal.configuration.validation.TestConfigurationValidator; +import org.apache.ignite.internal.eventlog.api.Event; +import org.apache.ignite.internal.eventlog.api.EventLog; import org.apache.ignite.internal.manager.ComponentContext; import org.apache.ignite.internal.network.configuration.NetworkExtensionConfigurationSchema; import org.apache.ignite.internal.rest.authentication.AuthenticationProviderFactory; @@ -98,7 +100,17 @@ public void setup() { Mockito.when(cmg.clusterState()).then(invocation -> CompletableFuture.completedFuture(state)); - AuthenticationManager authenticationManager = new AuthenticationManagerImpl(securityConfiguration, ign -> {}); + AuthenticationManager authenticationManager = new AuthenticationManagerImpl(securityConfiguration, new EventLog() { + @Override + public void log(Event event) { + + } + + @Override + public void log(String type, Supplier eventProvider) { + + } + }); Supplier authProviderFactory = () -> new AuthenticationProviderFactory(authenticationManager); Supplier restPresentationFactory = () -> new PresentationsFactory( configurationManager, diff --git a/modules/security/src/main/java/org/apache/ignite/internal/security/authentication/AuthenticationManagerImpl.java b/modules/security/src/main/java/org/apache/ignite/internal/security/authentication/AuthenticationManagerImpl.java index 153d1a9ea98..87fbc03c9cb 100644 --- a/modules/security/src/main/java/org/apache/ignite/internal/security/authentication/AuthenticationManagerImpl.java +++ b/modules/security/src/main/java/org/apache/ignite/internal/security/authentication/AuthenticationManagerImpl.java @@ -205,8 +205,9 @@ private CompletableFuture authenticate( } private void logAuthenticationFailure(AuthenticationRequest authenticationRequest) { - eventLog.log(() -> - IgniteEvents.USER_AUTHENTICATION_FAILURE.builder() + eventLog.log( + IgniteEvents.USER_AUTHENTICATION_FAILURE.type(), + () -> IgniteEvents.USER_AUTHENTICATION_FAILURE.builder() .user(EventUser.system()) .fields(Map.of("identity", tryGetUsernameOrUnknown(authenticationRequest))) .build() @@ -221,10 +222,12 @@ private static String tryGetUsernameOrUnknown(AuthenticationRequest authen } private void logUserAuthenticated(UserDetails userDetails) { - eventLog.log(() -> - IgniteEvents.USER_AUTHENTICATION_SUCCESS.create(EventUser.of( + eventLog.log( + IgniteEvents.USER_AUTHENTICATION_SUCCESS.type(), + () -> IgniteEvents.USER_AUTHENTICATION_SUCCESS.create(EventUser.of( userDetails.username(), userDetails.providerName() - ))); + )) + ); } private void refreshProviders(@Nullable SecurityView view) { diff --git a/modules/security/src/test/java/org/apache/ignite/internal/security/authentication/AuthenticationManagerImplTest.java b/modules/security/src/test/java/org/apache/ignite/internal/security/authentication/AuthenticationManagerImplTest.java index 0424a974658..c67d771158f 100644 --- a/modules/security/src/test/java/org/apache/ignite/internal/security/authentication/AuthenticationManagerImplTest.java +++ b/modules/security/src/test/java/org/apache/ignite/internal/security/authentication/AuthenticationManagerImplTest.java @@ -41,11 +41,14 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.CopyOnWriteArrayList; import java.util.function.Consumer; +import java.util.function.Supplier; import org.apache.ignite.configuration.annotation.ConfigurationType; import org.apache.ignite.internal.configuration.ClusterConfiguration; import org.apache.ignite.internal.configuration.testframework.ConfigurationExtension; import org.apache.ignite.internal.configuration.testframework.InjectConfiguration; import org.apache.ignite.internal.event.EventListener; +import org.apache.ignite.internal.eventlog.api.Event; +import org.apache.ignite.internal.eventlog.api.EventLog; import org.apache.ignite.internal.manager.ComponentContext; import org.apache.ignite.internal.security.authentication.basic.BasicAuthenticationProviderChange; import org.apache.ignite.internal.security.authentication.event.AuthenticationEvent; @@ -92,7 +95,17 @@ class AuthenticationManagerImplTest extends BaseIgniteAbstractTest { void setUp() { securityConfiguration = ((SecurityExtensionConfiguration) clusterConfiguration).security(); - manager = new AuthenticationManagerImpl(securityConfiguration, ign -> {}); + manager = new AuthenticationManagerImpl(securityConfiguration, new EventLog() { + @Override + public void log(Event event) { + + } + + @Override + public void log(String type, Supplier eventProvider) { + + } + }); Arrays.stream(AuthenticationEvent.values()).forEach(event -> manager.listen(event, listener)); From a9d3974f812e729f0706676b5d596790a9ed57b7 Mon Sep 17 00:00:00 2001 From: Pavel Pereslegin Date: Fri, 10 Jan 2025 16:45:28 +0300 Subject: [PATCH 008/113] IGNITE-23736 Sql. Provide ability to catch start/finish events for query (#4977) --- .../eventlog/api/IgniteEventType.java | 4 +- .../internal/eventlog/api/IgniteEvents.java | 2 + modules/runner/build.gradle | 1 + .../runner/app/ItIgniteNodeRestartTest.java | 17 +- .../ignite/internal/app/IgniteImpl.java | 3 +- modules/sql-engine/build.gradle | 4 + .../sql/engine/ItSqlQueryEventLogTest.java | 502 ++++++++++++++++++ .../sql/engine/QueryEventsFactory.java | 101 ++++ .../sql/engine/SqlOperationContext.java | 23 +- .../sql/engine/SqlQueryProcessor.java | 12 +- .../sql/engine/exec/ExecutionServiceImpl.java | 13 +- .../sql/engine/exec/TxAwareAsyncCursor.java | 18 +- .../exec/fsm/OptimizingPhaseHandler.java | 1 + .../internal/sql/engine/exec/fsm/Query.java | 12 + .../exec/fsm/QueryExecutionProgram.java | 2 + .../sql/engine/exec/fsm/QueryExecutor.java | 28 +- .../sql/engine/exec/fsm/QueryInfo.java | 13 + .../engine/exec/TxAwareCursorSelfTest.java | 9 +- .../sql/engine/framework/TestNode.java | 17 +- 19 files changed, 758 insertions(+), 24 deletions(-) create mode 100644 modules/sql-engine/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItSqlQueryEventLogTest.java create mode 100644 modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/QueryEventsFactory.java diff --git a/modules/eventlog/src/main/java/org/apache/ignite/internal/eventlog/api/IgniteEventType.java b/modules/eventlog/src/main/java/org/apache/ignite/internal/eventlog/api/IgniteEventType.java index c9d655a2220..3149261e06c 100644 --- a/modules/eventlog/src/main/java/org/apache/ignite/internal/eventlog/api/IgniteEventType.java +++ b/modules/eventlog/src/main/java/org/apache/ignite/internal/eventlog/api/IgniteEventType.java @@ -28,7 +28,9 @@ public enum IgniteEventType { USER_AUTHENTICATION_SUCCESS, USER_AUTHENTICATION_FAILURE, CLIENT_CONNECTION_ESTABLISHED, - CLIENT_CONNECTION_CLOSED; + CLIENT_CONNECTION_CLOSED, + QUERY_STARTED, + QUERY_FINISHED; static { // Without the following line, the IgniteEventType enum will not be registered in the EventTypeRegistry diff --git a/modules/eventlog/src/main/java/org/apache/ignite/internal/eventlog/api/IgniteEvents.java b/modules/eventlog/src/main/java/org/apache/ignite/internal/eventlog/api/IgniteEvents.java index 41d007bcd97..c6d62450f6b 100644 --- a/modules/eventlog/src/main/java/org/apache/ignite/internal/eventlog/api/IgniteEvents.java +++ b/modules/eventlog/src/main/java/org/apache/ignite/internal/eventlog/api/IgniteEvents.java @@ -34,6 +34,8 @@ public final class IgniteEvents implements EventFactory { public static final IgniteEvents CLIENT_CONNECTION_ESTABLISHED = new IgniteEvents(IgniteEventType.CLIENT_CONNECTION_ESTABLISHED); public static final IgniteEvents CLIENT_CONNECTION_CLOSED = new IgniteEvents(IgniteEventType.CLIENT_CONNECTION_CLOSED); + public static final IgniteEvents QUERY_STARTED = new IgniteEvents(IgniteEventType.QUERY_STARTED); + public static final IgniteEvents QUERY_FINISHED = new IgniteEvents(IgniteEventType.QUERY_FINISHED); private final IgniteEventType type; diff --git a/modules/runner/build.gradle b/modules/runner/build.gradle index 8f307c207ca..69efec98d94 100644 --- a/modules/runner/build.gradle +++ b/modules/runner/build.gradle @@ -175,6 +175,7 @@ dependencies { integrationTestImplementation project(':ignite-configuration-root') integrationTestImplementation project(':ignite-configuration-system') integrationTestImplementation project(':ignite-system-disaster-recovery') + integrationTestImplementation project(':ignite-eventlog') integrationTestImplementation testFixtures(project(":ignite-api")) integrationTestImplementation testFixtures(project(':ignite-core')) integrationTestImplementation testFixtures(project(':ignite-configuration')) diff --git a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/ItIgniteNodeRestartTest.java b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/ItIgniteNodeRestartTest.java index 9a221c55515..e9efcc2d8c9 100644 --- a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/ItIgniteNodeRestartTest.java +++ b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/ItIgniteNodeRestartTest.java @@ -125,6 +125,8 @@ import org.apache.ignite.internal.disaster.system.ClusterIdService; import org.apache.ignite.internal.disaster.system.SystemDisasterRecoveryStorage; import org.apache.ignite.internal.distributionzones.DistributionZoneManager; +import org.apache.ignite.internal.eventlog.api.Event; +import org.apache.ignite.internal.eventlog.api.EventLog; import org.apache.ignite.internal.failure.NoOpFailureManager; import org.apache.ignite.internal.hlc.ClockService; import org.apache.ignite.internal.hlc.ClockServiceImpl; @@ -772,6 +774,18 @@ public CompletableFuture> dataNodes(long causalityToken, int catalog lowWatermark ); + EventLog noopEventLog = new EventLog() { + @Override + public void log(Event event) { + // No-op. + } + + @Override + public void log(String type, Supplier eventProvider) { + // No-op. + } + }; + SqlQueryProcessor qryEngine = new SqlQueryProcessor( clusterSvc, logicalTopologyService, @@ -793,7 +807,8 @@ public CompletableFuture> dataNodes(long causalityToken, int catalog txManager, lowWatermark, threadPoolsManager.commonScheduler(), - new KillCommandHandler(name, logicalTopologyService, clusterSvc.messagingService()) + new KillCommandHandler(name, logicalTopologyService, clusterSvc.messagingService()), + noopEventLog ); sqlRef.set(new IgniteSqlImpl(qryEngine, HybridTimestampTracker.atomicTracker(null))); diff --git a/modules/runner/src/main/java/org/apache/ignite/internal/app/IgniteImpl.java b/modules/runner/src/main/java/org/apache/ignite/internal/app/IgniteImpl.java index aeea8fd4c74..c109175b9d2 100644 --- a/modules/runner/src/main/java/org/apache/ignite/internal/app/IgniteImpl.java +++ b/modules/runner/src/main/java/org/apache/ignite/internal/app/IgniteImpl.java @@ -1103,7 +1103,8 @@ public class IgniteImpl implements Ignite { txManager, lowWatermark, threadPoolsManager.commonScheduler(), - killCommandHandler + killCommandHandler, + eventLog ); systemViewManager.register(qryEngine); diff --git a/modules/sql-engine/build.gradle b/modules/sql-engine/build.gradle index 286d8a1c870..e161710c8ff 100644 --- a/modules/sql-engine/build.gradle +++ b/modules/sql-engine/build.gradle @@ -54,6 +54,7 @@ dependencies { implementation project(':ignite-partition-distribution') implementation project(':ignite-low-watermark') implementation project(':ignite-sql-engine-api') + implementation project(':ignite-eventlog') implementation libs.jetbrains.annotations implementation libs.fastutil.core implementation libs.caffeine @@ -133,6 +134,7 @@ dependencies { integrationTestImplementation project(':ignite-metrics') integrationTestImplementation project(":ignite-network-api") integrationTestImplementation project(':ignite-sql-engine-api') + integrationTestImplementation project(':ignite-eventlog') integrationTestImplementation testFixtures(project(':ignite-core')) integrationTestImplementation testFixtures(project(':ignite-schema')) integrationTestImplementation testFixtures(project(':ignite-sql-engine')) @@ -162,4 +164,6 @@ integrationTest { it.excludeTags "sqllogic" } } + + systemProperty "buildDirPath", project.buildDir.path } diff --git a/modules/sql-engine/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItSqlQueryEventLogTest.java b/modules/sql-engine/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItSqlQueryEventLogTest.java new file mode 100644 index 00000000000..d28e47de3ff --- /dev/null +++ b/modules/sql-engine/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItSqlQueryEventLogTest.java @@ -0,0 +1,502 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.sql.engine; + +import static org.apache.ignite.internal.sql.engine.util.SqlTestUtils.assertThrowsSqlException; +import static org.apache.ignite.internal.testframework.IgniteTestUtils.await; +import static org.apache.ignite.internal.testframework.IgniteTestUtils.runAsync; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.containsString; +import static org.hamcrest.Matchers.hasSize; +import static org.hamcrest.Matchers.is; +import static org.hamcrest.Matchers.notNullValue; +import static org.hamcrest.Matchers.nullValue; + +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.StandardOpenOption; +import java.util.ArrayList; +import java.util.List; +import java.util.UUID; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.Phaser; +import java.util.concurrent.TimeUnit; +import java.util.regex.Matcher; +import java.util.regex.Pattern; +import org.apache.ignite.InitParametersBuilder; +import org.apache.ignite.internal.eventlog.api.IgniteEventType; +import org.apache.ignite.internal.eventlog.api.IgniteEvents; +import org.apache.ignite.internal.eventlog.event.EventUser; +import org.apache.ignite.internal.lang.IgniteStringBuilder; +import org.apache.ignite.internal.properties.IgniteProductVersion; +import org.apache.ignite.internal.sql.BaseSqlIntegrationTest; +import org.apache.ignite.internal.sql.engine.QueryEventsFactory.FieldNames; +import org.apache.ignite.internal.tx.InternalTransaction; +import org.apache.ignite.internal.util.CompletableFutures; +import org.apache.ignite.lang.CancelHandle; +import org.apache.ignite.lang.CancellationToken; +import org.apache.ignite.lang.ErrorGroups.Sql; +import org.apache.ignite.lang.ErrorGroups.Transactions; +import org.apache.ignite.sql.ResultSet; +import org.apache.ignite.sql.SqlRow; +import org.apache.ignite.sql.Statement; +import org.apache.ignite.tx.Transaction; +import org.awaitility.Awaitility; +import org.jetbrains.annotations.Nullable; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +/** + * Integration tests to check the {@link IgniteEvents#QUERY_STARTED} and {@link IgniteEvents#QUERY_FINISHED} events. + */ +public class ItSqlQueryEventLogTest extends BaseSqlIntegrationTest { + private static final String UUID_PATTERN = "[0-9a-f]{8}-[0-9a-f]{4}-[0-9a-f]{4}-[0-9a-f]{4}-[0-9a-f]{12}"; + + private static Path eventlogPath; + + @BeforeAll + static void setUp() { + String buildDirPath = System.getProperty("buildDirPath"); + eventlogPath = Path.of(buildDirPath).resolve("event.log"); + + sql("CREATE TABLE test(id INT PRIMARY KEY)"); + } + + @BeforeEach + void cleanup() throws IOException { + List events = readEventLog(); + + sql("DELETE FROM test"); + + readEvents(events.size() + 2); + + resetLog(); + } + + @Override + protected int initialNodes() { + return 1; + } + + @Override + protected void configureInitParameters(InitParametersBuilder builder) { + String eventLog = "eventlog {\n" + + " sinks.logSink.channel: testChannel,\n" + + " channels.testChannel.events:" + + " [" + IgniteEventType.QUERY_STARTED + ", " + IgniteEventType.QUERY_FINISHED + "],\n" + + "}\n"; + + builder.clusterConfiguration("ignite {\n" + eventLog + "}"); + } + + @Test + void testQuery() { + String query = "SELECT 1"; + sql(query); + + List events = readEvents(2); + + UUID queryId = verifyQueryStartedFields(events.get(0), query, null); + verifyQueryFinishFields(events.get(1), queryId, SqlQueryType.QUERY, query, true, null); + } + + @Test + void testMultiStatementQuery() { + String scriptText = "INSERT INTO test VALUES (0); SELECT 1;"; + igniteSql().executeScript(scriptText); + + List events = readEvents(6); + + UUID scriptId = verifyQueryStartedFields(events.get(0), scriptText, null); + + String insertStatementText = "INSERT INTO `TEST`\\nVALUES ROW(0)"; + String selectStatementText = "SELECT 1"; + + UUID insertQueryId = verifyQueryStartedFields(events.get(1), insertStatementText, null, scriptId, 0); + + String selectStartEvent; + String insertFinishEvent; + + if (events.get(2).startsWith("{\"type\":\"QUERY_STARTED\"")) { + selectStartEvent = events.get(2); + insertFinishEvent = events.get(3); + } else { + selectStartEvent = events.get(3); + insertFinishEvent = events.get(2); + } + + verifyQueryFinishFields(insertFinishEvent, insertQueryId, SqlQueryType.DML, insertStatementText, null, true, + null, scriptId, 0); + + UUID selectQueryId = verifyQueryStartedFields(selectStartEvent, selectStatementText, null, scriptId, 1); + + // The order in which the last `SELECT` statement finish and script statement finish events are sent is undefined. + String selectFinishEvent; + String scriptFinishEvent; + + if (events.get(4).contains(selectQueryId.toString())) { + selectFinishEvent = events.get(4); + scriptFinishEvent = events.get(5); + } else { + selectFinishEvent = events.get(5); + scriptFinishEvent = events.get(4); + } + + verifyQueryFinishFields(selectFinishEvent, selectQueryId, SqlQueryType.QUERY, + selectStatementText, null, true, null, scriptId, 1); + + verifyQueryFinishFields(scriptFinishEvent, scriptId, null, + scriptText, null, false, null, null, -1); + } + + @Test + void testErrorResetOnTxRetry() throws IOException { + sql("CREATE TABLE my (id INT PRIMARY KEY, val INT)"); + sql("INSERT INTO my VALUES (1, 0), (2, 0), (3, 0), (4, 0)"); + + readEvents(4); + resetLog(); + + int parties = 2; + Phaser phaser = new Phaser(parties); + + List> results = new ArrayList<>(parties); + for (int i = 0; i < parties; i++) { + int newValue = i + 1; + results.add(runAsync(() -> { + phaser.awaitAdvanceInterruptibly(phaser.arrive()); + + sql("UPDATE my SET val = ?", newValue); + })); + } + + // all queries are expected to complete successfully + await(CompletableFutures.allOf(results)); + + List events = readEvents(4); + + { + FieldsChecker fieldsChecker = EventValidator.parseQueryFinish(events.get(2)); + fieldsChecker.verify(FieldNames.ERROR, null); + } + + { + FieldsChecker fieldsChecker = EventValidator.parseQueryFinish(events.get(3)); + fieldsChecker.verify(FieldNames.ERROR, null); + } + } + + @Test + public void testParseError() { + String query = "DELETE * FROM TEST"; + + assertThrowsSqlException( + Sql.STMT_PARSE_ERR, + "Failed to parse query: Encountered \"*\" at line 1, column 8", + () -> sql(query) + ); + + List events = readEvents(2); + + // QUERY_STARTED + UUID queryId = verifyQueryStartedFields(events.get(0), query, null); + + verifyQueryFinishFields(events.get(1), queryId, null, query, false, + "Failed to parse query: Encountered \\\"*\\\" at line 1, column 8"); + } + + @Test + public void testValidationError() { + String query = "INSERT INTO test VALUES (?), (?)"; + + String expErr = "Values passed to VALUES operator must have compatible types"; + + assertThrowsSqlException(Sql.STMT_VALIDATION_ERR, expErr, () -> sql(query, "1", 2)); + + List events = readEvents(2); + + UUID queryId = verifyQueryStartedFields(events.get(0), query, null); + verifyQueryFinishFields(events.get(1), queryId, SqlQueryType.DML, query, false, expErr); + } + + @Test + void testRuntimeError() { + String query = "SELECT 1 / ?"; + String expErr = "Division by zero"; + + assertThrowsSqlException(Sql.RUNTIME_ERR, expErr, () -> sql(query, 0)); + + List events = readEvents(2); + + UUID queryId = verifyQueryStartedFields(events.get(0), query, null); + verifyQueryFinishFields(events.get(1), queryId, SqlQueryType.QUERY, query, true, expErr); + } + + @Test + void testRuntimeError2() { + String query = "SELECT * FROM test WHERE id = (SELECT x FROM TABLE(SYSTEM_RANGE(1, 3)));"; + String expErr = "Subquery returned more than 1 value"; + + assertThrowsSqlException(Sql.RUNTIME_ERR, expErr, () -> sql(query)); + + List events = readEvents(2); + + UUID queryId = verifyQueryStartedFields(events.get(0), query, null); + verifyQueryFinishFields(events.get(1), queryId, SqlQueryType.QUERY, query, true, expErr); + } + + @Test + void testTxError() throws IOException { + Transaction tx = igniteTx().begin(); + sql(tx, "INSERT INTO test VALUES (0)"); + tx.commit(); + + readEvents(2); + resetLog(); + + String query = "INSERT INTO test VALUES (1)";; + String expErr = "Transaction is already finished"; + + assertThrowsSqlException(Transactions.TX_ALREADY_FINISHED_ERR, expErr, () -> igniteSql().execute(tx, query)); + + List events = readEvents(2); + + UUID queryId = verifyQueryStartedFields(events.get(0), query, tx); + verifyQueryFinishFields(events.get(1), queryId, SqlQueryType.DML, query, tx, false, expErr, null, -1); + } + + @Test + void testQueryCancelledError() { + Transaction tx = igniteTx().begin(); + + CancelHandle cancelHandle = CancelHandle.create(); + CancellationToken cancellationToken = cancelHandle.token(); + + String query = "SELECT x FROM TABLE(SYSTEM_RANGE(1, 100000))"; + + try (ResultSet ignore = igniteSql().execute(tx, cancellationToken, query)) { + List events = readEvents(1); + + UUID queryId = verifyQueryStartedFields(events.get(0), query, tx); + + cancelHandle.cancel(); + + events = readEvents(2); + + verifyQueryFinishFields(events.get(1), queryId, SqlQueryType.QUERY, query, true, QueryCancelledException.CANCEL_MSG); + } + } + + @Test + void testQueryTimeoutError() { + String query = "SELECT * FROM TABLE(SYSTEM_RANGE(1, 1000000000000000))"; + + Statement stmt = igniteSql().statementBuilder() + .query(query) + .queryTimeout(1, TimeUnit.MILLISECONDS) + .build(); + + assertThrowsSqlException( + Sql.EXECUTION_CANCELLED_ERR, + QueryCancelledException.TIMEOUT_MSG, + () -> igniteSql().execute(null, stmt) + ); + + List events = readEvents(2); + + UUID queryId = verifyQueryStartedFields(events.get(0), query, null); + verifyQueryFinishFields(events.get(1), queryId, SqlQueryType.QUERY, query, false, QueryCancelledException.TIMEOUT_MSG); + } + + @Test + void testDdlError() { + String query = "CREATE TABLE test (id INT PRIMARY KEY, val INT)"; + String expErr = "Table with name 'PUBLIC.TEST' already exists"; + + assertThrowsSqlException(Sql.STMT_VALIDATION_ERR, expErr, () -> sql(query)); + + List events = readEvents(2); + + UUID queryId = verifyQueryStartedFields(events.get(0), query, null); + verifyQueryFinishFields(events.get(1), queryId, SqlQueryType.DDL, query, false, expErr); + } + + private static List readEvents(int expectedCount) { + return Awaitility.await().timeout(5, TimeUnit.SECONDS).until(ItSqlQueryEventLogTest::readEventLog, hasSize(expectedCount)); + } + + private static List readEventLog() throws IOException { + return Files.readAllLines(eventlogPath); + } + + interface FieldsChecker { + /** Checks that the specified field has expected value. */ + void verify(String field, @Nullable Object val); + + /** + * Extracts field value using provided regular expression. + * + * @param field Field name. + * @param pattern Regular expression. + * @return Extracted field value. + * @throws AssertionError if the specified expression is not found. + */ + String matches(String field, String pattern); + } + + private static UUID verifyQueryStartedFields(String event, String expectedQueryText, @Nullable Transaction tx) { + return verifyQueryStartedFields(event, expectedQueryText, tx, null, -1); + } + + private static UUID verifyQueryStartedFields(String event, String expectedQueryText, @Nullable Transaction tx, @Nullable UUID parentId, + int statementNum) { + FieldsChecker fieldsChecker = EventValidator.parseQueryStart(event); + + fieldsChecker.verify(FieldNames.INITIATOR, CLUSTER.aliveNode().name()); + fieldsChecker.verify(FieldNames.SQL, expectedQueryText); + fieldsChecker.verify(FieldNames.SCHEMA, "PUBLIC"); + fieldsChecker.verify(FieldNames.PARENT_ID, parentId); + fieldsChecker.verify(FieldNames.STATEMENT_NUMBER, statementNum); + fieldsChecker.verify(FieldNames.TX_ID, tx == null ? null : ((InternalTransaction) tx).id()); + + String queryIdString = fieldsChecker.matches(FieldNames.ID, UUID_PATTERN); + + return UUID.fromString(queryIdString); + } + + private static void verifyQueryFinishFields(String event, UUID queryId, @Nullable SqlQueryType queryType, String expectedQueryText, + boolean checkImplicitTx, @Nullable String errMessage) { + verifyQueryFinishFields(event, queryId, queryType, expectedQueryText, null, checkImplicitTx, errMessage, null, -1); + } + + private static void verifyQueryFinishFields(String event, UUID queryId, SqlQueryType queryType, String expectedQueryText, + @Nullable Transaction tx, boolean checkImplicitTx, @Nullable String errMessage, @Nullable UUID parentId, int statementNum) { + FieldsChecker fieldsChecker = EventValidator.parseQueryFinish(event); + + fieldsChecker.verify("id", queryId); + fieldsChecker.verify(FieldNames.INITIATOR, CLUSTER.aliveNode().name()); + fieldsChecker.verify(FieldNames.SQL, expectedQueryText); + fieldsChecker.verify(FieldNames.SCHEMA, "PUBLIC"); + fieldsChecker.verify(FieldNames.PARENT_ID, parentId); + fieldsChecker.verify(FieldNames.STATEMENT_NUMBER, statementNum); + fieldsChecker.verify(FieldNames.TYPE, queryType == null ? null : queryType.name()); + fieldsChecker.matches(FieldNames.START_TIME, "\\d+"); + + if (checkImplicitTx) { + assertThat(tx, is(nullValue())); + + fieldsChecker.matches(FieldNames.TX_ID, UUID_PATTERN); + } else { + fieldsChecker.verify(FieldNames.TX_ID, tx == null ? null : ((InternalTransaction) tx).id()); + } + + if (errMessage == null) { + fieldsChecker.verify(FieldNames.ERROR, null); + } else { + fieldsChecker.matches(FieldNames.ERROR, ".*" + Pattern.quote(errMessage) + ".*"); + } + } + + private static void resetLog() throws IOException { + Files.write(eventlogPath, new byte[0], StandardOpenOption.TRUNCATE_EXISTING); + } + + private static class EventValidator { + private static final String USER = EventUser.system().username(); + + private static final String PROVIDER = EventUser.system().authenticationProvider(); + + private static final Pattern QUERY_START_PATTERN = Pattern.compile("\\{" + + "\"type\":\"QUERY_STARTED\"," + + "\"timestamp\":\\d+," + + "\"productVersion\":\"" + IgniteProductVersion.VERSION_PATTERN.pattern() + "\"," + + "\"user\":\\{\"username\":\"" + USER + "\",\"authenticationProvider\":\"" + PROVIDER + "\"}," + + "\"fields\":\\{(?.+)}" + + "}"); + + private static final Pattern QUERY_FINISH_PATTERN = Pattern.compile("\\{" + + "\"type\":\"QUERY_FINISHED\"," + + "\"timestamp\":\\d+," + + "\"productVersion\":\"" + IgniteProductVersion.VERSION_PATTERN.pattern() + "\"," + + "\"user\":\\{\"username\":\"" + USER + "\",\"authenticationProvider\":\"" + PROVIDER + "\"}," + + "\"fields\":\\{(?.+)}" + + "}"); + + static FieldsChecker parseQueryStart(String jsonString) { + Matcher matcher = QUERY_START_PATTERN.matcher(jsonString); + + assertThat(matcher.matches(), is(true)); + + return new FieldsCheckerImpl(matcher.group("fields")); + } + + static FieldsChecker parseQueryFinish(String jsonString) { + Matcher matcher = QUERY_FINISH_PATTERN.matcher(jsonString); + + assertThat("input=" + jsonString, matcher.matches(), is(true)); + + return new FieldsCheckerImpl(matcher.group("fields")); + } + + private static class FieldsCheckerImpl implements FieldsChecker { + private final String fields; + + FieldsCheckerImpl(String fields) { + this.fields = fields; + } + + @Override + public void verify(String field, Object val) { + IgniteStringBuilder subj = new IgniteStringBuilder('"').app(field).app('"'); + + subj.app(':'); + + if (val == null) { + subj.app("null"); + } else if (val instanceof String || val instanceof UUID) { + subj.app('"').app(val).app('"'); + } else { + subj.app(val); + } + + assertThat(fields, containsString(subj.toString())); + } + + @Override + public String matches(String field, String regex) { + IgniteStringBuilder fullRegexp = new IgniteStringBuilder() + .app('"').app(field).app('"') + .app(":\"?(").app(regex).app(")\"?"); + + Pattern pattern = Pattern.compile(fullRegexp.toString()); + + Matcher matcher = pattern.matcher(fields); + + assertThat("pattern=" + pattern.pattern() + ", fields=" + fields, matcher.find(), is(true)); + + String result = matcher.group(1); + + assertThat(result, is(notNullValue())); + + return result; + } + } + } +} diff --git a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/QueryEventsFactory.java b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/QueryEventsFactory.java new file mode 100644 index 00000000000..576f88101a2 --- /dev/null +++ b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/QueryEventsFactory.java @@ -0,0 +1,101 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.sql.engine; + +import static org.apache.ignite.internal.util.ExceptionUtils.unwrapCause; + +import java.util.Map; +import org.apache.ignite.internal.eventlog.api.Event; +import org.apache.ignite.internal.eventlog.api.IgniteEvents; +import org.apache.ignite.internal.eventlog.event.EventUser; +import org.apache.ignite.internal.sql.engine.exec.fsm.QueryInfo; +import org.apache.ignite.internal.util.IgniteUtils; + +/** + * SQL query event factory. + */ +public class QueryEventsFactory { + private final String localNode; + + public QueryEventsFactory(String localNode) { + this.localNode = localNode; + } + + /** Creates new {@link IgniteEvents#QUERY_STARTED} event. */ + public Event makeStartEvent(QueryInfo queryInfo, EventUser user) { + Map fields = IgniteUtils.newLinkedHashMap(7); + + fillCommonFields(fields, queryInfo); + + return IgniteEvents.QUERY_STARTED.builder() + .user(user) + .timestamp(queryInfo.startTime().toEpochMilli()) + .fields(fields) + .build(); + } + + /** Creates new {@link IgniteEvents#QUERY_FINISHED} event. */ + public Event makeFinishEvent(QueryInfo queryInfo, EventUser user, long finishTime) { + Map fields = IgniteUtils.newLinkedHashMap(10); + + fillCommonFields(fields, queryInfo); + + fields.put(FieldNames.START_TIME, queryInfo.startTime().toEpochMilli()); + + SqlQueryType queryType = queryInfo.queryType(); + + fields.put(FieldNames.TYPE, queryType == null ? null : queryType.name()); + + Throwable error = queryInfo.error(); + + fields.put(FieldNames.ERROR, error == null ? null : unwrapCause(error).getMessage()); + + return IgniteEvents.QUERY_FINISHED.builder() + .user(user) + .timestamp(finishTime) + .fields(fields) + .build(); + } + + private void fillCommonFields(Map fields, QueryInfo queryInfo) { + fields.put(FieldNames.INITIATOR, localNode); + fields.put(FieldNames.ID, queryInfo.id()); + fields.put(FieldNames.SCHEMA, queryInfo.schema()); + fields.put(FieldNames.SQL, queryInfo.sql()); + fields.put(FieldNames.PARENT_ID, queryInfo.parentId()); + fields.put(FieldNames.STATEMENT_NUMBER, queryInfo.statementNum()); + fields.put(FieldNames.TX_ID, queryInfo.transactionId()); + } + + /** Query events field names. */ + static class FieldNames { + // Common fields. + static final String INITIATOR = "initiatorNode"; + static final String ID = "id"; + static final String SCHEMA = "schema"; + static final String SQL = "sql"; + static final String PARENT_ID = "parentId"; + static final String STATEMENT_NUMBER = "statementNum"; + static final String TX_ID = "transactionId"; + + // Finish event fields. + static final String START_TIME = "startTime"; + static final String TYPE = "type"; + static final String ERROR = "errorMessage"; + } +} diff --git a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/SqlOperationContext.java b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/SqlOperationContext.java index fc59e9029eb..8710e7779b0 100644 --- a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/SqlOperationContext.java +++ b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/SqlOperationContext.java @@ -49,6 +49,7 @@ public final class SqlOperationContext { private final @Nullable QueryCancel cancel; private final @Nullable String defaultSchemaName; private final @Nullable Consumer txUsedListener; + private final @Nullable Consumer errorListener; /** * Private constructor, used by a builder. @@ -61,7 +62,8 @@ private SqlOperationContext( @Nullable QueryTransactionContext txContext, @Nullable QueryCancel cancel, @Nullable String defaultSchemaName, - @Nullable Consumer txUsedListener + @Nullable Consumer txUsedListener, + @Nullable Consumer errorListener ) { this.queryId = queryId; this.timeZoneId = timeZoneId; @@ -71,6 +73,7 @@ private SqlOperationContext( this.cancel = cancel; this.defaultSchemaName = defaultSchemaName; this.txUsedListener = txUsedListener; + this.errorListener = errorListener; } public static Builder builder() { @@ -128,6 +131,15 @@ public void notifyTxUsed(QueryTransactionWrapper tx) { } } + /** + * Notifies the context that an error has occurred. + */ + public void notifyError(Throwable th) { + if (errorListener != null) { + errorListener.accept(th); + } + } + /** * Returns the operation time. * @@ -162,6 +174,7 @@ public static class Builder { private @Nullable QueryTransactionContext txContext; private @Nullable Consumer txUsedListener; + private @Nullable Consumer errorListener; private @Nullable QueryCancel cancel; private @Nullable String defaultSchemaName; @@ -205,6 +218,11 @@ public Builder txUsedListener(Consumer txUsedListener) return this; } + public Builder errorHandler(Consumer errorListener) { + this.errorListener = errorListener; + return this; + } + /** Creates new context. */ public SqlOperationContext build() { return new SqlOperationContext( @@ -215,7 +233,8 @@ public SqlOperationContext build() { txContext, cancel, defaultSchemaName, - txUsedListener + txUsedListener, + errorListener ); } } diff --git a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/SqlQueryProcessor.java b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/SqlQueryProcessor.java index 94aa7386667..47bc58ad50a 100644 --- a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/SqlQueryProcessor.java +++ b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/SqlQueryProcessor.java @@ -39,6 +39,7 @@ import java.util.function.LongSupplier; import org.apache.ignite.internal.catalog.CatalogManager; import org.apache.ignite.internal.cluster.management.topology.api.LogicalTopologyService; +import org.apache.ignite.internal.eventlog.api.EventLog; import org.apache.ignite.internal.failure.FailureManager; import org.apache.ignite.internal.hlc.ClockService; import org.apache.ignite.internal.hlc.HybridTimestamp; @@ -208,6 +209,8 @@ public class SqlQueryProcessor implements QueryProcessor, SystemViewProvider { private final ScheduledExecutorService commonScheduler; + private final EventLog eventLog; + /** Constructor. */ public SqlQueryProcessor( ClusterService clusterSrvc, @@ -230,7 +233,8 @@ public SqlQueryProcessor( TxManager txManager, LowWatermark lowWaterMark, ScheduledExecutorService commonScheduler, - KillCommandHandler killCommandHandler + KillCommandHandler killCommandHandler, + EventLog eventLog ) { this.clusterSrvc = clusterSrvc; this.logicalTopologyService = logicalTopologyService; @@ -252,6 +256,8 @@ public SqlQueryProcessor( this.txManager = txManager; this.commonScheduler = commonScheduler; this.killCommandHandler = killCommandHandler; + this.eventLog = eventLog; + sqlStatisticManager = new SqlStatisticManagerImpl(tableManager, catalogManager, lowWaterMark); sqlSchemaManager = new SqlSchemaManagerImpl( catalogManager, @@ -352,6 +358,7 @@ public synchronized CompletableFuture startAsync(ComponentContext componen )); queryExecutor = registerService(new QueryExecutor( + clusterSrvc.topologyService().localMember().name(), CACHE_FACTORY, PARSED_RESULT_CACHE_SIZE, new ParserServiceImpl(), @@ -364,7 +371,8 @@ public synchronized CompletableFuture startAsync(ComponentContext componen executionSrvc, DEFAULT_PROPERTIES, txTracker, - new QueryIdGenerator(nodeName.hashCode()) + new QueryIdGenerator(nodeName.hashCode()), + eventLog )); queriesViewProvider.init(queryExecutor); diff --git a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/ExecutionServiceImpl.java b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/ExecutionServiceImpl.java index 8cb9ad0885d..26fbf233d28 100644 --- a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/ExecutionServiceImpl.java +++ b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/ExecutionServiceImpl.java @@ -362,7 +362,8 @@ private CompletableFuture> executeQuery( txWrapper, dataCursor, firstPageReady0, - queryManager::close + queryManager::close, + operationContext::notifyError )); return f.whenComplete((r, t) -> { @@ -448,12 +449,9 @@ private AsyncDataCursor executeExecutablePlan( assert txContext != null; - QueryTransactionWrapper txWrapper = txContext.explicitTx(); + QueryTransactionWrapper txWrapper = txContext.getOrStartSqlManaged(((ExplainablePlan) plan).type() != SqlQueryType.DML, true); - if (txWrapper == null) { - // Underlying table will drive transaction by itself. - txWrapper = txContext.getOrStartSqlManaged(((ExplainablePlan) plan).type() != SqlQueryType.DML, true); - } + operationContext.notifyTxUsed(txWrapper); PrefetchCallback prefetchCallback = new PrefetchCallback(); @@ -471,7 +469,8 @@ private AsyncDataCursor executeExecutablePlan( txWrapper, dataCursor, prefetchCallback.prefetchFuture(), - reason -> nullCompletedFuture() + reason -> nullCompletedFuture(), + operationContext::notifyError ); } diff --git a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/TxAwareAsyncCursor.java b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/TxAwareAsyncCursor.java index db0779e6301..372fdc8010b 100644 --- a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/TxAwareAsyncCursor.java +++ b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/TxAwareAsyncCursor.java @@ -22,6 +22,7 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.CompletionException; import java.util.concurrent.atomic.AtomicBoolean; +import java.util.function.Consumer; import java.util.function.Function; import org.apache.ignite.internal.lang.SqlExceptionMapperUtil; import org.apache.ignite.internal.sql.engine.QueryCancelledException; @@ -43,17 +44,20 @@ class TxAwareAsyncCursor implements AsyncDataCursorExt { private final AsyncCursor dataCursor; private final CompletableFuture firstPageReady; private final Function> closeHandler; + private final Consumer errorListener; TxAwareAsyncCursor( QueryTransactionWrapper txWrapper, AsyncCursor dataCursor, CompletableFuture firstPageReady, - Function> closeHandler + Function> closeHandler, + Consumer errorListener ) { this.txWrapper = txWrapper; this.dataCursor = dataCursor; this.firstPageReady = firstPageReady; this.closeHandler = closeHandler; + this.errorListener = errorListener; } @Override @@ -82,11 +86,15 @@ public CompletableFuture cancelAsync(CancellationReason reason) { closeHandler.apply(reason) .thenCompose(ignored -> { if (reason != CancellationReason.CLOSE) { - String message = reason == CancellationReason.TIMEOUT - ? QueryCancelledException.TIMEOUT_MSG + String message = reason == CancellationReason.TIMEOUT + ? QueryCancelledException.TIMEOUT_MSG : QueryCancelledException.CANCEL_MSG; - return txWrapper.rollback(new QueryCancelledException(message)); + QueryCancelledException cancelEx = new QueryCancelledException(message); + + errorListener.accept(cancelEx); + + return txWrapper.rollback(cancelEx); } return txWrapper.commitImplicit(); @@ -126,6 +134,8 @@ public CompletableFuture onFirstPageReady() { private CompletableFuture handleError(Throwable throwable) { Throwable wrapped = wrapIfNecessary(throwable); + errorListener.accept(throwable); + return txWrapper.rollback(throwable) .handle((none, rollbackError) -> { if (rollbackError != null) { diff --git a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/fsm/OptimizingPhaseHandler.java b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/fsm/OptimizingPhaseHandler.java index 639fb332f0a..486a18a0e43 100644 --- a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/fsm/OptimizingPhaseHandler.java +++ b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/fsm/OptimizingPhaseHandler.java @@ -64,6 +64,7 @@ public Result handle(Query query) { .operationTime(operationTime) .txContext(query.txContext) .txUsedListener(tx -> query.usedTransaction = tx) + .errorHandler(throwable -> query.setError(throwable)) .build(); query.operationContext = operationContext; diff --git a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/fsm/Query.java b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/fsm/Query.java index 9a45b261f64..caf1bfc1d78 100644 --- a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/fsm/Query.java +++ b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/fsm/Query.java @@ -23,6 +23,7 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.atomic.AtomicReference; import org.apache.ignite.internal.sql.engine.AsyncSqlCursor; import org.apache.ignite.internal.sql.engine.InternalSqlRow; import org.apache.ignite.internal.sql.engine.QueryCancel; @@ -53,6 +54,7 @@ class Query { final QueryExecutor executor; final SqlProperties properties; final QueryTransactionContext txContext; + final AtomicReference error = new AtomicReference<>(); final @Nullable CompletableFuture> nextCursorFuture; // Below is volatile state populated during processing of particular stage for single statement execution @@ -133,11 +135,21 @@ ExecutionPhase currentPhase() { } void onError(Throwable th) { + setError(th); + moveTo(ExecutionPhase.TERMINATED); resultHolder.completeExceptionally(th); } + void setError(Throwable err) { + Throwable prevErr = error.compareAndExchange(null, err); + + if (prevErr != null && prevErr != err) { + error.get().addSuppressed(err); + } + } + /** * Cancels this query. * diff --git a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/fsm/QueryExecutionProgram.java b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/fsm/QueryExecutionProgram.java index 7500df7b595..5721d9b7289 100644 --- a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/fsm/QueryExecutionProgram.java +++ b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/fsm/QueryExecutionProgram.java @@ -70,6 +70,8 @@ private QueryExecutionProgram() { static boolean errorHandler(Query query, Throwable th) { if (canRecover(query, th)) { + query.error.set(null); + if (nodeLeft(th)) { SqlOperationContext context = query.operationContext; diff --git a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/fsm/QueryExecutor.java b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/fsm/QueryExecutor.java index 91e23f490ae..85679a99bc7 100644 --- a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/fsm/QueryExecutor.java +++ b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/fsm/QueryExecutor.java @@ -28,6 +28,9 @@ import java.util.stream.Collectors; import org.apache.ignite.internal.catalog.Catalog; import org.apache.ignite.internal.catalog.CatalogService; +import org.apache.ignite.internal.eventlog.api.EventLog; +import org.apache.ignite.internal.eventlog.api.IgniteEventType; +import org.apache.ignite.internal.eventlog.event.EventUser; import org.apache.ignite.internal.hlc.ClockService; import org.apache.ignite.internal.hlc.HybridTimestamp; import org.apache.ignite.internal.lang.NodeStoppingException; @@ -35,6 +38,7 @@ import org.apache.ignite.internal.sql.engine.AsyncSqlCursor; import org.apache.ignite.internal.sql.engine.InternalSqlRow; import org.apache.ignite.internal.sql.engine.QueryCancelledException; +import org.apache.ignite.internal.sql.engine.QueryEventsFactory; import org.apache.ignite.internal.sql.engine.QueryProperty; import org.apache.ignite.internal.sql.engine.SqlOperationContext; import org.apache.ignite.internal.sql.engine.exec.AsyncDataCursorExt; @@ -81,9 +85,14 @@ public class QueryExecutor implements LifecycleAware { private final ConcurrentMap runningQueries = new ConcurrentHashMap<>(); + private final EventLog eventLog; + + private final QueryEventsFactory eventsFactory; + /** * Creates executor. * + * @param nodeId Local node consistent ID. * @param cacheFactory Factory to create cache for parsed AST. * @param parsedResultsCacheSize Size of the cache for parsed AST. * @param parserService Service to parse query string. @@ -97,8 +106,10 @@ public class QueryExecutor implements LifecycleAware { * @param defaultProperties Set of properties to use as defaults. * @param transactionTracker Tracker to track usage of transactions by query. * @param idGenerator Id generator used to provide cluster-wide unique query id. + * @param eventLog Event log. */ public QueryExecutor( + String nodeId, CacheFactory cacheFactory, int parsedResultsCacheSize, ParserService parserService, @@ -111,7 +122,8 @@ public QueryExecutor( ExecutionService executionService, SqlProperties defaultProperties, TransactionTracker transactionTracker, - QueryIdGenerator idGenerator + QueryIdGenerator idGenerator, + EventLog eventLog ) { this.queryToParsedResultCache = cacheFactory.create(parsedResultsCacheSize); this.parserService = parserService; @@ -125,6 +137,8 @@ public QueryExecutor( this.defaultProperties = defaultProperties; this.transactionTracker = transactionTracker; this.idGenerator = idGenerator; + this.eventLog = eventLog; + this.eventsFactory = new QueryEventsFactory(nodeId); } /** @@ -318,11 +332,21 @@ MultiStatementHandler createScriptHandler(Query query) { private void trackQuery(Query query, @Nullable CancellationToken cancellationToken) { Query old = runningQueries.put(query.id, query); + eventLog.log(IgniteEventType.QUERY_STARTED.name(), + () -> eventsFactory.makeStartEvent(new QueryInfo(query), EventUser.system())); + assert old == null : "Query with the same id already registered"; CompletableFuture queryTerminationFut = query.onPhaseStarted(ExecutionPhase.TERMINATED); - queryTerminationFut.whenComplete((ignored, ex) -> runningQueries.remove(query.id)); + queryTerminationFut.whenComplete((none, ignoredEx) -> { + runningQueries.remove(query.id); + + long finishTime = clockService.current().getPhysical(); + + eventLog.log(IgniteEventType.QUERY_FINISHED.name(), + () -> eventsFactory.makeFinishEvent(new QueryInfo(query), EventUser.system(), finishTime)); + }); if (cancellationToken != null) { CancelHandleHelper.addCancelAction(cancellationToken, query::cancel, queryTerminationFut); diff --git a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/fsm/QueryInfo.java b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/fsm/QueryInfo.java index 8f7b655d32a..009425fa413 100644 --- a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/fsm/QueryInfo.java +++ b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/exec/fsm/QueryInfo.java @@ -37,6 +37,7 @@ public class QueryInfo { private final @Nullable UUID transactionId; private final int statementNum; private final boolean script; + private final @Nullable Throwable error; QueryInfo(Query query) { id = query.id; @@ -48,6 +49,7 @@ public class QueryInfo { phase = query.currentPhase(); queryType = deriveQueryType(query.parsedResult); transactionId = deriveTxId(query); + error = query.error.get(); script = query.parsedScript != null; } @@ -57,6 +59,12 @@ public class QueryInfo { } private static @Nullable UUID deriveTxId(Query query) { + QueryTransactionWrapper explicit = query.txContext.explicitTx(); + + if (explicit != null) { + return explicit.unwrap().id(); + } + QueryTransactionWrapper tx = query.usedTransaction; return tx != null ? tx.unwrap().id() : null; @@ -111,4 +119,9 @@ public ExecutionPhase phase() { public int statementNum() { return statementNum; } + + /** Returns an error if one occurred during execution and caused the query to terminate. */ + public @Nullable Throwable error() { + return error; + } } diff --git a/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/exec/TxAwareCursorSelfTest.java b/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/exec/TxAwareCursorSelfTest.java index e5e66516ab2..3c33722c6d4 100644 --- a/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/exec/TxAwareCursorSelfTest.java +++ b/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/exec/TxAwareCursorSelfTest.java @@ -58,7 +58,8 @@ public void testTriggerCommitAfterDataIsFullyRead(boolean implicit, QueryTransac txWrapper, new AsyncWrapper<>(CompletableFuture.completedFuture(list.iterator()), Runnable::run), nullCompletedFuture(), - reason -> nullCompletedFuture() + reason -> nullCompletedFuture(), + ex -> {} ); int requestRows = 2; @@ -85,7 +86,8 @@ public void testExceptionRollbacksImplicitTx(boolean implicit, QueryTransactionW txWrapper, new AsyncWrapper<>(CompletableFuture.failedFuture(err), Runnable::run), nullCompletedFuture(), - reason -> nullCompletedFuture() + reason -> nullCompletedFuture(), + ex -> {} ); CompletionException t = assertThrows(CompletionException.class, () -> cursor.requestNextAsync(1).join()); @@ -106,7 +108,8 @@ public void testCloseCommitsImplicitTx(boolean implicit, QueryTransactionWrapper txWrapper, data, nullCompletedFuture(), - reason -> nullCompletedFuture() + reason -> nullCompletedFuture(), + ex -> {} ); cursor.closeAsync().join(); diff --git a/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/framework/TestNode.java b/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/framework/TestNode.java index ab8bf237756..375ca7f2a23 100644 --- a/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/framework/TestNode.java +++ b/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/framework/TestNode.java @@ -30,8 +30,11 @@ import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; import java.util.function.Function; +import java.util.function.Supplier; import java.util.stream.Collectors; import org.apache.ignite.internal.catalog.CatalogService; +import org.apache.ignite.internal.eventlog.api.Event; +import org.apache.ignite.internal.eventlog.api.EventLog; import org.apache.ignite.internal.failure.FailureContext; import org.apache.ignite.internal.failure.FailureManager; import org.apache.ignite.internal.failure.handlers.AbstractFailureHandler; @@ -200,6 +203,7 @@ public void stop() { }); queryExecutor = registerService(new QueryExecutor( + nodeName, EmptyCacheFactory.INSTANCE, 0, parserService, @@ -212,7 +216,18 @@ public void stop() { executionService, SqlQueryProcessor.DEFAULT_PROPERTIES, NoOpTransactionTracker.INSTANCE, - new QueryIdGenerator(nodeName.hashCode()) + new QueryIdGenerator(nodeName.hashCode()), + new EventLog() { + @Override + public void log(Event event) { + // No-op. + } + + @Override + public void log(String type, Supplier eventProvider) { + // No-op. + } + } )); } From e676ea5b072b83285b86e737c4fa1d769467d713 Mon Sep 17 00:00:00 2001 From: Mikhail Date: Fri, 10 Jan 2025 16:57:59 +0300 Subject: [PATCH 009/113] IGNITE-24080 Fix config path reading in CLI init command (#5001) --- .../cli/commands/cluster/init/ClusterInitOptions.java | 4 ++-- .../cli/commands/cluster/init/ClusterInitReplCommand.java | 6 ++++-- 2 files changed, 6 insertions(+), 4 deletions(-) diff --git a/modules/cli/src/main/java/org/apache/ignite/internal/cli/commands/cluster/init/ClusterInitOptions.java b/modules/cli/src/main/java/org/apache/ignite/internal/cli/commands/cluster/init/ClusterInitOptions.java index 0d0d5f9619a..92cb903aaff 100644 --- a/modules/cli/src/main/java/org/apache/ignite/internal/cli/commands/cluster/init/ClusterInitOptions.java +++ b/modules/cli/src/main/java/org/apache/ignite/internal/cli/commands/cluster/init/ClusterInitOptions.java @@ -201,8 +201,8 @@ private static boolean tryParseConfig(String config) { private static boolean checkConfigAsPath(String config) { try { - Paths.get(config); - return true; + Path path = Paths.get(config); + return Files.exists(path); } catch (InvalidPathException e) { return false; } diff --git a/modules/cli/src/main/java/org/apache/ignite/internal/cli/commands/cluster/init/ClusterInitReplCommand.java b/modules/cli/src/main/java/org/apache/ignite/internal/cli/commands/cluster/init/ClusterInitReplCommand.java index 0cf94f8effb..a3715cb237d 100644 --- a/modules/cli/src/main/java/org/apache/ignite/internal/cli/commands/cluster/init/ClusterInitReplCommand.java +++ b/modules/cli/src/main/java/org/apache/ignite/internal/cli/commands/cluster/init/ClusterInitReplCommand.java @@ -17,6 +17,7 @@ package org.apache.ignite.internal.cli.commands.cluster.init; +import static org.apache.ignite.internal.cli.commands.Options.Constants.CLUSTER_CONFIG_OPTION; import static org.apache.ignite.internal.cli.core.style.component.QuestionUiComponent.fromYesNoQuestion; import static picocli.CommandLine.Command; @@ -66,8 +67,9 @@ private FlowBuilder askQuestionIfConfigIsPath() { return Flows.from(this::buildCallInput); } catch (ConfigAsPathException e) { QuestionUiComponent questionUiComponent = fromYesNoQuestion( - "It seems that you passed the path to the config file to the config content option. " - + "Do you want this file to be read as a config?" + "It seems that you have passed the path to the configuration file in the configuration content " + + CLUSTER_CONFIG_OPTION + " option. " + + "Do you want to read cluster configuration from this file?" ); return Flows.acceptQuestion(questionUiComponent, From 7eb75ec7088ac3bcdb9e9cc8e8b1aea5b632612d Mon Sep 17 00:00:00 2001 From: Alexander Polovtcev Date: Fri, 10 Jan 2025 17:24:32 +0200 Subject: [PATCH 010/113] IGNITE-24014 Introduce a way to have key-value entries in configuration (#4997) --- examples/config/ignite-config.conf | 4 +- .../topology/ItLogicalTopologyTest.java | 2 +- .../NodeAttributeConfigurationSchema.java | 4 +- .../ItConfigurationProcessorTest.java | 60 ++++++ ...ipleInjectedValuesConfigurationSchema.java | 30 +++ ...supportedFieldTypeConfigurationSchema.java | 27 +++ .../ValidConfigurationSchema.java | 31 +++ ...ueAndInjectedValueConfigurationSchema.java | 34 +++ .../processor/ConfigurationProcessor.java | 71 ++++--- .../ConfigurationProcessorUtils.java | 22 +- .../validators/InjectedValueValidator.java | 101 +++++++++ .../annotation/InjectedValue.java | 79 +++++++ .../SystemPropertyConfigurationSchema.java | 4 +- ...ibutedConfigurationPropertyHolderTest.java | 3 +- .../asm/ConfigurationImplAsmGenerator.java | 3 +- .../asm/InnerNodeAsmGenerator.java | 26 ++- .../hocon/HoconListConfigurationSource.java | 44 ++-- .../hocon/HoconObjectConfigurationSource.java | 92 +++++---- .../HoconPrimitiveConfigurationSource.java | 14 +- .../tree/ConstructableTreeNode.java | 9 + .../tree/ConverterToMapVisitor.java | 71 +++++-- .../configuration/util/ConfigurationUtil.java | 23 ++- .../InjectedValueConfigurationTest.java | 194 ++++++++++++++++++ .../hocon/HoconConverterTest.java | 2 +- .../ItDistributionZonesFiltersTest.java | 20 +- .../ItRebalanceTriggersRecoveryTest.java | 4 +- .../BaseDistributionZoneManagerTest.java | 4 +- .../distribution-zones/tech-notes/filters.md | 15 +- .../metastorage/TestMetasStorageUtils.java | 4 +- ...ageCompactionTriggerConfigurationTest.java | 4 +- .../replicator/ItReplicaLifecycleTest.java | 6 +- .../runner/app/ItIgniteNodeRestartTest.java | 4 +- .../runner/app/ItReplicaStateManagerTest.java | 6 +- .../sql/engine/ItUnstableTopologyTest.java | 2 +- ...ePartitionsRecoveryByFilterUpdateTest.java | 6 +- 35 files changed, 843 insertions(+), 182 deletions(-) create mode 100644 modules/configuration-annotation-processor/src/integrationTest/resources/org/apache/ignite/internal/configuration/processor/injectedvalue/MultipleInjectedValuesConfigurationSchema.java create mode 100644 modules/configuration-annotation-processor/src/integrationTest/resources/org/apache/ignite/internal/configuration/processor/injectedvalue/UnsupportedFieldTypeConfigurationSchema.java create mode 100644 modules/configuration-annotation-processor/src/integrationTest/resources/org/apache/ignite/internal/configuration/processor/injectedvalue/ValidConfigurationSchema.java create mode 100644 modules/configuration-annotation-processor/src/integrationTest/resources/org/apache/ignite/internal/configuration/processor/injectedvalue/ValueAndInjectedValueConfigurationSchema.java create mode 100644 modules/configuration-annotation-processor/src/main/java/org/apache/ignite/internal/configuration/processor/validators/InjectedValueValidator.java create mode 100644 modules/configuration-api/src/main/java/org/apache/ignite/configuration/annotation/InjectedValue.java create mode 100644 modules/configuration/src/test/java/org/apache/ignite/internal/configuration/InjectedValueConfigurationTest.java diff --git a/examples/config/ignite-config.conf b/examples/config/ignite-config.conf index adc047156ab..6f95edcdcb6 100644 --- a/examples/config/ignite-config.conf +++ b/examples/config/ignite-config.conf @@ -24,7 +24,7 @@ ignite { ] } nodeAttributes.nodeAttributes { - region.attribute = US - storage.attribute = SSD + region = US + storage = SSD } } diff --git a/modules/cluster-management/src/integrationTest/java/org/apache/ignite/internal/cluster/management/topology/ItLogicalTopologyTest.java b/modules/cluster-management/src/integrationTest/java/org/apache/ignite/internal/cluster/management/topology/ItLogicalTopologyTest.java index 3a7593d302d..97e3816272a 100644 --- a/modules/cluster-management/src/integrationTest/java/org/apache/ignite/internal/cluster/management/topology/ItLogicalTopologyTest.java +++ b/modules/cluster-management/src/integrationTest/java/org/apache/ignite/internal/cluster/management/topology/ItLogicalTopologyTest.java @@ -67,7 +67,7 @@ class ItLogicalTopologyTest extends ClusterPerTestIntegrationTest { + " port: {},\n" + " nodeFinder.netClusterNodes: [ {} ]\n" + " },\n" - + " nodeAttributes.nodeAttributes: {region.attribute = US, storage.attribute = SSD},\n" + + " nodeAttributes.nodeAttributes: {region = US, storage = SSD},\n" + " storage.profiles: {lru_rocks.engine = rocksdb, segmented_aipersist.engine = aipersist},\n" + " clientConnector.port: {},\n" + " rest.port: {},\n" diff --git a/modules/cluster-management/src/main/java/org/apache/ignite/internal/cluster/management/configuration/NodeAttributeConfigurationSchema.java b/modules/cluster-management/src/main/java/org/apache/ignite/internal/cluster/management/configuration/NodeAttributeConfigurationSchema.java index cb52ebf3a2e..3dc822dc4a4 100644 --- a/modules/cluster-management/src/main/java/org/apache/ignite/internal/cluster/management/configuration/NodeAttributeConfigurationSchema.java +++ b/modules/cluster-management/src/main/java/org/apache/ignite/internal/cluster/management/configuration/NodeAttributeConfigurationSchema.java @@ -19,7 +19,7 @@ import org.apache.ignite.configuration.annotation.Config; import org.apache.ignite.configuration.annotation.InjectedName; -import org.apache.ignite.configuration.annotation.Value; +import org.apache.ignite.configuration.annotation.InjectedValue; /** * Node's attribute configuration schema. User can specify any number of pairs (key, attribute) for a node through the local configuration @@ -35,6 +35,6 @@ public class NodeAttributeConfigurationSchema { public String name; /** Node attribute field. */ - @Value(hasDefault = true) + @InjectedValue(hasDefault = true) public String attribute = ""; } diff --git a/modules/configuration-annotation-processor/src/integrationTest/java/org/apache/ignite/internal/configuration/processor/ItConfigurationProcessorTest.java b/modules/configuration-annotation-processor/src/integrationTest/java/org/apache/ignite/internal/configuration/processor/ItConfigurationProcessorTest.java index b47f017e453..ce2f61768cc 100644 --- a/modules/configuration-annotation-processor/src/integrationTest/java/org/apache/ignite/internal/configuration/processor/ItConfigurationProcessorTest.java +++ b/modules/configuration-annotation-processor/src/integrationTest/java/org/apache/ignite/internal/configuration/processor/ItConfigurationProcessorTest.java @@ -615,6 +615,66 @@ void testSuccessfulCodeGenerationAbstractConfigurationAndItsDescendants() { configRootConfigurationInterfaceContent.contains("extends " + getConfigurationInterfaceName(abstractRootConfigSchema).simpleName()); } + @Test + void testSuccessInjectedValueFieldCodeGeneration() { + String packageName = "org.apache.ignite.internal.configuration.processor.injectedvalue"; + + ClassName cls0 = ClassName.get(packageName, "ValidConfigurationSchema"); + + BatchCompilation batchCompile = batchCompile(cls0); + + assertThat(batchCompile.getCompilationStatus()).succeededWithoutWarnings(); + + assertEquals(3, batchCompile.generated().size()); + + assertTrue(batchCompile.getBySchema(cls0).allGenerated()); + } + + @Test + void testMultipleInjectedValuesUnsuccessfulGeneration() { + String packageName = "org.apache.ignite.internal.configuration.processor.injectedvalue"; + + ClassName cls0 = ClassName.get(packageName, "MultipleInjectedValuesConfigurationSchema"); + + assertThrowsEx( + IllegalStateException.class, + () -> batchCompile(cls0), + "Field marked as @InjectedValue must be the only \"value\" field in the schema " + + "org.apache.ignite.internal.configuration.processor.injectedvalue.MultipleInjectedValuesConfigurationSchema, " + + "found: [firstValue, secondValue]" + ); + } + + @Test + void testValuesAndInjectedValueUnsuccessfulGeneration() { + String packageName = "org.apache.ignite.internal.configuration.processor.injectedvalue"; + + ClassName cls0 = ClassName.get(packageName, "ValueAndInjectedValueConfigurationSchema"); + + assertThrowsEx( + IllegalStateException.class, + () -> batchCompile(cls0), + "Field marked as @InjectedValue must be the only \"value\" field in the schema " + + "org.apache.ignite.internal.configuration.processor.injectedvalue.ValueAndInjectedValueConfigurationSchema, " + + "found: [secondValue, firstValue, thirdValue]" + ); + } + + @Test + void testUnsupportedFieldTypeUnsuccessfulGeneration() { + String packageName = "org.apache.ignite.internal.configuration.processor.injectedvalue"; + + ClassName cls0 = ClassName.get(packageName, "UnsupportedFieldTypeConfigurationSchema"); + + assertThrowsEx( + IllegalStateException.class, + () -> batchCompile(cls0), + "org.apache.ignite.internal.configuration.processor.injectedvalue.UnsupportedFieldTypeConfigurationSchema.firstValue " + + "field must have one of the following types: boolean, int, long, double, String, UUID " + + "or an array of aforementioned type." + ); + } + /** * Compile set of classes. * diff --git a/modules/configuration-annotation-processor/src/integrationTest/resources/org/apache/ignite/internal/configuration/processor/injectedvalue/MultipleInjectedValuesConfigurationSchema.java b/modules/configuration-annotation-processor/src/integrationTest/resources/org/apache/ignite/internal/configuration/processor/injectedvalue/MultipleInjectedValuesConfigurationSchema.java new file mode 100644 index 00000000000..02d7994e83a --- /dev/null +++ b/modules/configuration-annotation-processor/src/integrationTest/resources/org/apache/ignite/internal/configuration/processor/injectedvalue/MultipleInjectedValuesConfigurationSchema.java @@ -0,0 +1,30 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.configuration.processor.injectedvalue; + +import org.apache.ignite.configuration.annotation.Config; +import org.apache.ignite.configuration.annotation.InjectedValue; + +@Config +public class MultipleInjectedValuesConfigurationSchema { + @InjectedValue + public String firstValue; + + @InjectedValue + public String secondValue; +} diff --git a/modules/configuration-annotation-processor/src/integrationTest/resources/org/apache/ignite/internal/configuration/processor/injectedvalue/UnsupportedFieldTypeConfigurationSchema.java b/modules/configuration-annotation-processor/src/integrationTest/resources/org/apache/ignite/internal/configuration/processor/injectedvalue/UnsupportedFieldTypeConfigurationSchema.java new file mode 100644 index 00000000000..e869d08f031 --- /dev/null +++ b/modules/configuration-annotation-processor/src/integrationTest/resources/org/apache/ignite/internal/configuration/processor/injectedvalue/UnsupportedFieldTypeConfigurationSchema.java @@ -0,0 +1,27 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.configuration.processor.injectedvalue; + +import org.apache.ignite.configuration.annotation.Config; +import org.apache.ignite.configuration.annotation.InjectedValue; + +@Config +public class UnsupportedFieldTypeConfigurationSchema { + @InjectedValue + public Object firstValue; +} diff --git a/modules/configuration-annotation-processor/src/integrationTest/resources/org/apache/ignite/internal/configuration/processor/injectedvalue/ValidConfigurationSchema.java b/modules/configuration-annotation-processor/src/integrationTest/resources/org/apache/ignite/internal/configuration/processor/injectedvalue/ValidConfigurationSchema.java new file mode 100644 index 00000000000..0698a0768a9 --- /dev/null +++ b/modules/configuration-annotation-processor/src/integrationTest/resources/org/apache/ignite/internal/configuration/processor/injectedvalue/ValidConfigurationSchema.java @@ -0,0 +1,31 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.configuration.processor.injectedvalue; + +import org.apache.ignite.configuration.annotation.Config; +import org.apache.ignite.configuration.annotation.InjectedName; +import org.apache.ignite.configuration.annotation.InjectedValue; + +@Config +public class ValidConfigurationSchema { + @InjectedName + public String name; + + @InjectedValue + public String someValue; +} diff --git a/modules/configuration-annotation-processor/src/integrationTest/resources/org/apache/ignite/internal/configuration/processor/injectedvalue/ValueAndInjectedValueConfigurationSchema.java b/modules/configuration-annotation-processor/src/integrationTest/resources/org/apache/ignite/internal/configuration/processor/injectedvalue/ValueAndInjectedValueConfigurationSchema.java new file mode 100644 index 00000000000..f1e42d03563 --- /dev/null +++ b/modules/configuration-annotation-processor/src/integrationTest/resources/org/apache/ignite/internal/configuration/processor/injectedvalue/ValueAndInjectedValueConfigurationSchema.java @@ -0,0 +1,34 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.configuration.processor.injectedvalue; + +import org.apache.ignite.configuration.annotation.Config; +import org.apache.ignite.configuration.annotation.InjectedValue; +import org.apache.ignite.configuration.annotation.Value; + +@Config +public class ValueAndInjectedValueConfigurationSchema { + @Value + public String firstValue; + + @InjectedValue + public String secondValue; + + @Value + public String thirdValue; +} diff --git a/modules/configuration-annotation-processor/src/main/java/org/apache/ignite/internal/configuration/processor/ConfigurationProcessor.java b/modules/configuration-annotation-processor/src/main/java/org/apache/ignite/internal/configuration/processor/ConfigurationProcessor.java index 76e630a6b75..3dc7f7007a2 100644 --- a/modules/configuration-annotation-processor/src/main/java/org/apache/ignite/internal/configuration/processor/ConfigurationProcessor.java +++ b/modules/configuration-annotation-processor/src/main/java/org/apache/ignite/internal/configuration/processor/ConfigurationProcessor.java @@ -24,6 +24,7 @@ import static javax.lang.model.element.Modifier.PUBLIC; import static javax.lang.model.element.Modifier.STATIC; import static org.apache.ignite.internal.configuration.processor.ConfigurationProcessorUtils.collectFieldsWithAnnotation; +import static org.apache.ignite.internal.configuration.processor.ConfigurationProcessorUtils.containsAnyAnnotation; import static org.apache.ignite.internal.configuration.processor.ConfigurationProcessorUtils.findFirstPresentAnnotation; import static org.apache.ignite.internal.configuration.processor.ConfigurationProcessorUtils.getChangeName; import static org.apache.ignite.internal.configuration.processor.ConfigurationProcessorUtils.getConfigurationInterfaceName; @@ -80,6 +81,7 @@ import org.apache.ignite.configuration.annotation.ConfigurationExtension; import org.apache.ignite.configuration.annotation.ConfigurationRoot; import org.apache.ignite.configuration.annotation.InjectedName; +import org.apache.ignite.configuration.annotation.InjectedValue; import org.apache.ignite.configuration.annotation.InternalId; import org.apache.ignite.configuration.annotation.NamedConfigValue; import org.apache.ignite.configuration.annotation.PolymorphicConfig; @@ -87,6 +89,7 @@ import org.apache.ignite.configuration.annotation.PolymorphicId; import org.apache.ignite.configuration.annotation.Secret; import org.apache.ignite.configuration.annotation.Value; +import org.apache.ignite.internal.configuration.processor.validators.InjectedValueValidator; import org.jetbrains.annotations.Nullable; /** @@ -153,12 +156,16 @@ private boolean process0(RoundEnvironment roundEnvironment) { return false; } + var injectedValueValidator = new InjectedValueValidator(processingEnv); + for (TypeElement clazz : annotatedConfigs) { // Find all the fields of the schema. List fields = fields(clazz); validateConfigurationSchemaClass(clazz, fields); + injectedValueValidator.validate(clazz, fields); + // Get package name of the schema class String packageName = elementUtils.getPackageOf(clazz).getQualifiedName().toString(); @@ -175,10 +182,7 @@ private boolean process0(RoundEnvironment roundEnvironment) { throw new ConfigurationProcessorException("Field " + clazz.getQualifiedName() + "." + field + " must be public"); } - final String fieldName = field.getSimpleName().toString(); - - // Get configuration types (VIEW, CHANGE and so on) - final TypeName interfaceGetMethodType = getInterfaceGetMethodType(field); + String fieldName = field.getSimpleName().toString(); if (field.getAnnotation(ConfigValue.class) != null) { checkConfigField(field, ConfigValue.class); @@ -190,8 +194,7 @@ private boolean process0(RoundEnvironment roundEnvironment) { checkConfigField(field, NamedConfigValue.class); } - Value valueAnnotation = field.getAnnotation(Value.class); - if (valueAnnotation != null) { + if (field.getAnnotation(Value.class) != null) { // Must be a primitive or an array of the primitives (including java.lang.String, java.util.UUID). if (!isValidValueAnnotationFieldType(field.asType())) { throw new ConfigurationProcessorException(String.format( @@ -204,8 +207,7 @@ private boolean process0(RoundEnvironment roundEnvironment) { } } - PolymorphicId polymorphicId = field.getAnnotation(PolymorphicId.class); - if (polymorphicId != null) { + if (field.getAnnotation(PolymorphicId.class) != null) { if (!isClass(field.asType(), String.class)) { throw new ConfigurationProcessorException(String.format( FIELD_MUST_BE_SPECIFIC_CLASS_ERROR_FORMAT, @@ -237,6 +239,9 @@ private boolean process0(RoundEnvironment roundEnvironment) { } } + // Get configuration types (VIEW, CHANGE and so on) + TypeName interfaceGetMethodType = getInterfaceGetMethodType(field); + createGetters(configurationInterfaceBuilder, fieldName, interfaceGetMethodType); } @@ -355,13 +360,11 @@ private static void createGetters( * @return Bundle with all types for configuration */ private static TypeName getInterfaceGetMethodType(VariableElement field) { - TypeName interfaceGetMethodType = null; - TypeName baseType = TypeName.get(field.asType()); ConfigValue confAnnotation = field.getAnnotation(ConfigValue.class); if (confAnnotation != null) { - interfaceGetMethodType = getConfigurationInterfaceName((ClassName) baseType); + return getConfigurationInterfaceName((ClassName) baseType); } NamedConfigValue namedConfigAnnotation = field.getAnnotation(NamedConfigValue.class); @@ -371,7 +374,7 @@ private static TypeName getInterfaceGetMethodType(VariableElement field) { TypeName viewClassType = getViewName((ClassName) baseType); TypeName changeClassType = getChangeName((ClassName) baseType); - interfaceGetMethodType = ParameterizedTypeName.get( + return ParameterizedTypeName.get( ClassName.get(NamedConfigurationTree.class), interfaceGetType, viewClassType, @@ -379,13 +382,16 @@ private static TypeName getInterfaceGetMethodType(VariableElement field) { ); } - Value valueAnnotation = field.getAnnotation(Value.class); - PolymorphicId polymorphicIdAnnotation = field.getAnnotation(PolymorphicId.class); - InjectedName injectedNameAnnotation = field.getAnnotation(InjectedName.class); - InternalId internalIdAnnotation = field.getAnnotation(InternalId.class); + boolean containsAnnotation = containsAnyAnnotation( + field, + Value.class, + PolymorphicId.class, + InjectedName.class, + InternalId.class, + InjectedValue.class + ); - if (valueAnnotation != null || polymorphicIdAnnotation != null || injectedNameAnnotation != null - || internalIdAnnotation != null) { + if (containsAnnotation) { // It is necessary to use class names without loading classes so that we won't // accidentally get NoClassDefFoundError ClassName confValueClass = ClassName.get("org.apache.ignite.configuration", "ConfigurationValue"); @@ -396,10 +402,10 @@ private static TypeName getInterfaceGetMethodType(VariableElement field) { genericType = genericType.box(); } - interfaceGetMethodType = ParameterizedTypeName.get(confValueClass, genericType); + return ParameterizedTypeName.get(confValueClass, genericType); } - return interfaceGetMethodType; + throw new IllegalArgumentException(String.format("Field \"%s\" does not contain any supported annotations", field)); } /** @@ -503,8 +509,6 @@ private void createPojoBindings( ClassName consumerClsName = ClassName.get(Consumer.class); for (VariableElement field : fields) { - Value valAnnotation = field.getAnnotation(Value.class); - String fieldName = field.getSimpleName().toString(); TypeMirror schemaFieldType = field.asType(); TypeName schemaFieldTypeName = TypeName.get(schemaFieldType); @@ -512,15 +516,13 @@ private void createPojoBindings( boolean leafField = isValidValueAnnotationFieldType(schemaFieldType) || !((ClassName) schemaFieldTypeName).simpleName().contains(CONFIGURATION_SCHEMA_POSTFIX); - boolean namedListField = field.getAnnotation(NamedConfigValue.class) != null; - TypeName viewFieldType = leafField ? schemaFieldTypeName : getViewName((ClassName) schemaFieldTypeName); TypeName changeFieldType = leafField ? schemaFieldTypeName : getChangeName((ClassName) schemaFieldTypeName); - if (namedListField) { + if (field.getAnnotation(NamedConfigValue.class) != null) { changeFieldType = ParameterizedTypeName.get( ClassName.get(NamedListChange.class), viewFieldType, @@ -540,8 +542,7 @@ private void createPojoBindings( viewClsBuilder.addMethod(getMtdBuilder.build()); // Read only. - if (field.getAnnotation(PolymorphicId.class) != null || field.getAnnotation(InjectedName.class) != null - || field.getAnnotation(InternalId.class) != null) { + if (containsAnyAnnotation(field, PolymorphicId.class, InjectedName.class, InternalId.class)) { continue; } @@ -551,7 +552,7 @@ private void createPojoBindings( .addModifiers(PUBLIC, ABSTRACT) .returns(changeClsName); - if (valAnnotation != null) { + if (containsAnyAnnotation(field, Value.class, InjectedValue.class)) { if (schemaFieldType.getKind() == TypeKind.ARRAY) { changeMtdBuilder.varargs(true); } @@ -559,18 +560,16 @@ private void createPojoBindings( changeMtdBuilder.addParameter(changeFieldType, fieldName); } else { changeMtdBuilder.addParameter(ParameterizedTypeName.get(consumerClsName, changeFieldType), fieldName); - } - changeClsBuilder.addMethod(changeMtdBuilder.build()); - - // Create "FooChange changeFoo()" method with no parameters, if it's a config value or named list value. - if (valAnnotation == null) { + // Create "FooChange changeFoo()" method with no parameters, if it's a config value or named list value. MethodSpec.Builder shortChangeMtdBuilder = MethodSpec.methodBuilder(changeMtdName) .addModifiers(PUBLIC, ABSTRACT) .returns(changeFieldType); changeClsBuilder.addMethod(shortChangeMtdBuilder.build()); } + + changeClsBuilder.addMethod(changeMtdBuilder.build()); } if (isPolymorphicConfig) { @@ -1112,8 +1111,8 @@ private void validateNameFields(TypeElement clazz, List fields) * Checks for missing {@link org.apache.ignite.configuration.annotation.Name} for nested schema with {@link InjectedName}. * * @param field Class field. - * @throws ConfigurationProcessorException If there is no {@link org.apache.ignite.configuration.annotation.Name} for the nested schema - * with {@link InjectedName}. + * @throws ConfigurationProcessorException If there is no {@link org.apache.ignite.configuration.annotation.Name} for the nested + * schema with {@link InjectedName}. */ private void checkMissingNameForInjectedName(VariableElement field) { TypeElement fieldType = (TypeElement) processingEnv.getTypeUtils().asElement(field.asType()); @@ -1123,7 +1122,7 @@ private void checkMissingNameForInjectedName(VariableElement field) { List fields; if (!isClass(superClassFieldType.asType(), Object.class) - && findFirstPresentAnnotation(superClassFieldType, AbstractConfiguration.class).isPresent()) { + && superClassFieldType.getAnnotation(AbstractConfiguration.class) != null) { fields = concat( collectFieldsWithAnnotation(fields(fieldType), InjectedName.class), collectFieldsWithAnnotation(fields(superClassFieldType), InjectedName.class) diff --git a/modules/configuration-annotation-processor/src/main/java/org/apache/ignite/internal/configuration/processor/ConfigurationProcessorUtils.java b/modules/configuration-annotation-processor/src/main/java/org/apache/ignite/internal/configuration/processor/ConfigurationProcessorUtils.java index 5b3a2e70974..5061557ed15 100644 --- a/modules/configuration-annotation-processor/src/main/java/org/apache/ignite/internal/configuration/processor/ConfigurationProcessorUtils.java +++ b/modules/configuration-annotation-processor/src/main/java/org/apache/ignite/internal/configuration/processor/ConfigurationProcessorUtils.java @@ -27,13 +27,13 @@ import java.util.Objects; import java.util.Optional; import java.util.stream.Stream; -import javax.lang.model.element.TypeElement; +import javax.lang.model.element.Element; import javax.lang.model.element.VariableElement; /** * Annotation processing utilities. */ -class ConfigurationProcessorUtils { +public class ConfigurationProcessorUtils { /** * Returns {@link ClassName} for configuration class public interface. * @@ -91,17 +91,25 @@ public static String joinSimpleName(String delimiter, Class findFirstPresentAnnotation( - TypeElement clazz, + Element element, Class... annotationClasses ) { - return Stream.of(annotationClasses).map(clazz::getAnnotation).filter(Objects::nonNull).findFirst(); + return Stream.of(annotationClasses).map(element::getAnnotation).filter(Objects::nonNull).findFirst(); + } + + /** + * Returns {@code true} if any of the given annotations are present on the given element. + */ + @SafeVarargs + public static boolean containsAnyAnnotation(Element element, Class... annotationClasses) { + return findFirstPresentAnnotation(element, annotationClasses).isPresent(); } /** diff --git a/modules/configuration-annotation-processor/src/main/java/org/apache/ignite/internal/configuration/processor/validators/InjectedValueValidator.java b/modules/configuration-annotation-processor/src/main/java/org/apache/ignite/internal/configuration/processor/validators/InjectedValueValidator.java new file mode 100644 index 00000000000..7963ff81242 --- /dev/null +++ b/modules/configuration-annotation-processor/src/main/java/org/apache/ignite/internal/configuration/processor/validators/InjectedValueValidator.java @@ -0,0 +1,101 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.configuration.processor.validators; + +import static org.apache.ignite.internal.configuration.processor.ConfigurationProcessorUtils.collectFieldsWithAnnotation; +import static org.apache.ignite.internal.configuration.processor.ConfigurationProcessorUtils.simpleName; +import static org.apache.ignite.internal.util.CollectionUtils.concat; + +import java.util.List; +import java.util.UUID; +import javax.annotation.processing.ProcessingEnvironment; +import javax.lang.model.element.TypeElement; +import javax.lang.model.element.VariableElement; +import javax.lang.model.type.ArrayType; +import javax.lang.model.type.TypeKind; +import javax.lang.model.type.TypeMirror; +import org.apache.ignite.configuration.annotation.InjectedValue; +import org.apache.ignite.configuration.annotation.Value; +import org.apache.ignite.internal.configuration.processor.ConfigurationProcessorException; + +/** + * Validator class for the {@link InjectedValue} annotation. + */ +public class InjectedValueValidator { + private final ProcessingEnvironment processingEnv; + + public InjectedValueValidator(ProcessingEnvironment processingEnv) { + this.processingEnv = processingEnv; + } + + /** + * Validates invariants of the {@link InjectedValue} annotation. This includes: + * + *
    + *
  1. Type of InjectedValue field is either a primitive, or a String, or a UUID;
  2. + *
  3. There is only a single InjectedValue field in the schema (including {@link Value} fields).
  4. + *
+ */ + public void validate(TypeElement clazz, List fields) { + List injectedValueFields = collectFieldsWithAnnotation(fields, InjectedValue.class); + + if (injectedValueFields.isEmpty()) { + return; + } + + List valueFields = collectFieldsWithAnnotation(fields, Value.class); + + if (injectedValueFields.size() > 1 || !valueFields.isEmpty()) { + throw new ConfigurationProcessorException(String.format( + "Field marked as %s must be the only \"value\" field in the schema %s, found: %s", + simpleName(InjectedValue.class), + clazz.getQualifiedName(), + concat(injectedValueFields, valueFields) + )); + } + + VariableElement injectedValueField = injectedValueFields.get(0); + + // Must be a primitive or an array of the primitives (including java.lang.String, java.util.UUID). + if (!isValidValueAnnotationFieldType(injectedValueField.asType())) { + throw new ConfigurationProcessorException(String.format( + "%s.%s field must have one of the following types: " + + "boolean, int, long, double, String, UUID or an array of aforementioned type.", + clazz.getQualifiedName(), + injectedValueField.getSimpleName() + )); + } + } + + private boolean isValidValueAnnotationFieldType(TypeMirror type) { + if (type.getKind() == TypeKind.ARRAY) { + type = ((ArrayType) type).getComponentType(); + } + + return type.getKind().isPrimitive() || isClass(type, String.class) || isClass(type, UUID.class); + } + + private boolean isClass(TypeMirror type, Class clazz) { + TypeMirror classType = processingEnv + .getElementUtils() + .getTypeElement(clazz.getCanonicalName()) + .asType(); + + return classType.equals(type); + } +} diff --git a/modules/configuration-api/src/main/java/org/apache/ignite/configuration/annotation/InjectedValue.java b/modules/configuration-api/src/main/java/org/apache/ignite/configuration/annotation/InjectedValue.java new file mode 100644 index 00000000000..403f100a8e8 --- /dev/null +++ b/modules/configuration-api/src/main/java/org/apache/ignite/configuration/annotation/InjectedValue.java @@ -0,0 +1,79 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.configuration.annotation; + +import static java.lang.annotation.ElementType.FIELD; +import static java.lang.annotation.RetentionPolicy.RUNTIME; + +import java.lang.annotation.Documented; +import java.lang.annotation.Retention; +import java.lang.annotation.Target; + +/** + * This annotations is intended to be placed on {@link NamedConfigValue} elements to emulate key-value pairs in configuration schemas. + * + *

Therefore, this annotation induces the following constraints: + * + *

    + *
  1. Must be placed on a field of a configuration schema that represents a Named List element;
  2. + *
  3. Must be the only field in the configuration schema, apart from the one marked with {@link InjectedName}.
  4. + *
+ * + *

In all other aspects it behaves exactly like the {@link Value} annotation. + * + *

For example, this annotation can be used to declare a configuration schema with arbitrary {@code String} properties: + * + *

{@code
+ *     @Config
+ *     class PropertyConfigurationSchema {
+ *         @NamedConfigValue
+ *         public PropertyEntryConfigurationSchema properties;
+ *     }
+ *
+ *     @Config
+ *     class PropertyEntryConfigurationSchema {
+ *         @InjectedName
+ *         public String propertyName;
+ *
+ *         @InjectedValue
+ *         public String propertyValue;
+ *     }
+ * }
+ * + *

This will allow to use the following HOCON to represent this configuration: + * + *

{@code
+ *     root.properties {
+ *         property1: "value1",
+ *         property2: "value2",
+ *         property3: "value3"
+ *     }
+ * }
+ */ +@Target(FIELD) +@Retention(RUNTIME) +@Documented +public @interface InjectedValue { + /** + * Indicates that the current configuration value has a default value. Value itself is derived from the instantiated object of a + * corresponding schema type. This means that the default is not necessarily a constant value. + * + * @return {@code hasDefault} flag value. + */ + boolean hasDefault() default false; +} diff --git a/modules/configuration-system/src/main/java/org/apache/ignite/internal/configuration/SystemPropertyConfigurationSchema.java b/modules/configuration-system/src/main/java/org/apache/ignite/internal/configuration/SystemPropertyConfigurationSchema.java index acbe11a4e2b..ad7a6c08c94 100644 --- a/modules/configuration-system/src/main/java/org/apache/ignite/internal/configuration/SystemPropertyConfigurationSchema.java +++ b/modules/configuration-system/src/main/java/org/apache/ignite/internal/configuration/SystemPropertyConfigurationSchema.java @@ -20,7 +20,7 @@ import org.apache.ignite.configuration.ConfigurationModule; import org.apache.ignite.configuration.annotation.Config; import org.apache.ignite.configuration.annotation.InjectedName; -import org.apache.ignite.configuration.annotation.Value; +import org.apache.ignite.configuration.annotation.InjectedValue; import org.apache.ignite.configuration.validation.CamelCaseKeys; import org.apache.ignite.internal.configuration.validation.LongNumberSystemPropertyValueValidator; @@ -41,6 +41,6 @@ public class SystemPropertyConfigurationSchema { @InjectedName public String name; - @Value + @InjectedValue public String propertyValue; } diff --git a/modules/configuration-system/src/test/java/org/apache/ignite/internal/configuration/utils/SystemDistributedConfigurationPropertyHolderTest.java b/modules/configuration-system/src/test/java/org/apache/ignite/internal/configuration/utils/SystemDistributedConfigurationPropertyHolderTest.java index b9a9b600a05..96d99299077 100644 --- a/modules/configuration-system/src/test/java/org/apache/ignite/internal/configuration/utils/SystemDistributedConfigurationPropertyHolderTest.java +++ b/modules/configuration-system/src/test/java/org/apache/ignite/internal/configuration/utils/SystemDistributedConfigurationPropertyHolderTest.java @@ -60,8 +60,7 @@ void testEmptySystemProperties(@InjectConfiguration SystemDistributedConfigurati @Test void testValidSystemPropertiesOnStart( - @InjectConfiguration("mock.properties = {" - + PROPERTY_NAME + ".propertyValue = \"newValue\"}") + @InjectConfiguration("mock.properties." + PROPERTY_NAME + " = newValue") SystemDistributedConfiguration systemConfig ) { var config = noopConfigHolder(systemConfig); diff --git a/modules/configuration/src/main/java/org/apache/ignite/internal/configuration/asm/ConfigurationImplAsmGenerator.java b/modules/configuration/src/main/java/org/apache/ignite/internal/configuration/asm/ConfigurationImplAsmGenerator.java index a4a2d2fab42..8c0d45f8ef8 100644 --- a/modules/configuration/src/main/java/org/apache/ignite/internal/configuration/asm/ConfigurationImplAsmGenerator.java +++ b/modules/configuration/src/main/java/org/apache/ignite/internal/configuration/asm/ConfigurationImplAsmGenerator.java @@ -54,6 +54,7 @@ import static org.apache.ignite.internal.configuration.util.ConfigurationUtil.isPolymorphicConfig; import static org.apache.ignite.internal.configuration.util.ConfigurationUtil.isPolymorphicConfigInstance; import static org.apache.ignite.internal.configuration.util.ConfigurationUtil.isPolymorphicId; +import static org.apache.ignite.internal.configuration.util.ConfigurationUtil.isReadOnly; import static org.apache.ignite.internal.configuration.util.ConfigurationUtil.isValue; import static org.apache.ignite.internal.configuration.util.ConfigurationUtil.polymorphicInstanceId; import static org.apache.ignite.internal.util.ArrayUtils.nullOrEmpty; @@ -385,7 +386,7 @@ private void addConfigurationImplConstructor( rootKeyVar, changerVar, listenOnlyVar, - constantBoolean(isPolymorphicId(schemaField) || isInjectedName(schemaField) || isInternalId(schemaField)) + constantBoolean(isReadOnly(schemaField)) ); } else { SchemaClassesInfo fieldInfo = cgen.schemaInfo(schemaField.getType()); diff --git a/modules/configuration/src/main/java/org/apache/ignite/internal/configuration/asm/InnerNodeAsmGenerator.java b/modules/configuration/src/main/java/org/apache/ignite/internal/configuration/asm/InnerNodeAsmGenerator.java index 506a34b602e..d1841df1a53 100644 --- a/modules/configuration/src/main/java/org/apache/ignite/internal/configuration/asm/InnerNodeAsmGenerator.java +++ b/modules/configuration/src/main/java/org/apache/ignite/internal/configuration/asm/InnerNodeAsmGenerator.java @@ -59,10 +59,12 @@ import static org.apache.ignite.internal.configuration.util.ConfigurationUtil.hasDefault; import static org.apache.ignite.internal.configuration.util.ConfigurationUtil.isConfigValue; import static org.apache.ignite.internal.configuration.util.ConfigurationUtil.isInjectedName; +import static org.apache.ignite.internal.configuration.util.ConfigurationUtil.isInjectedValue; import static org.apache.ignite.internal.configuration.util.ConfigurationUtil.isNamedConfigValue; import static org.apache.ignite.internal.configuration.util.ConfigurationUtil.isPolymorphicConfig; import static org.apache.ignite.internal.configuration.util.ConfigurationUtil.isPolymorphicConfigInstance; import static org.apache.ignite.internal.configuration.util.ConfigurationUtil.isPolymorphicId; +import static org.apache.ignite.internal.configuration.util.ConfigurationUtil.isReadOnly; import static org.apache.ignite.internal.configuration.util.ConfigurationUtil.isValue; import static org.apache.ignite.internal.configuration.util.ConfigurationUtil.polymorphicInstanceId; import static org.apache.ignite.internal.util.CollectionUtils.concat; @@ -177,6 +179,9 @@ class InnerNodeAsmGenerator extends AbstractAsmGenerator { /** {@link ConstructableTreeNode#construct(String, ConfigurationSource, boolean)} method name. */ private static final String CONSTRUCT_MTD_NAME = "construct"; + /** {@link ConstructableTreeNode#injectedValueFieldName}. */ + private static final String INJECTED_VALUE_FIELD_NAME_MTD_NAME = "injectedValueFieldName"; + /** Mapping for each configuration {@link Field} to a static constant with this {@link Field} as value. */ private final Map fieldToFieldDefinitionMap = new HashMap<>(); @@ -309,6 +314,9 @@ private ClassDefinition createNodeClass() { // Field with @InjectedName. FieldDefinition injectedNameFieldDef = null; + // Field with @InjectedValue. + Field injectedValueField = null; + for (Field schemaField : concat(schemaFields, publicExtensionFields, internalExtensionFields, polymorphicFields)) { FieldDefinition fieldDef = addInnerNodeField(schemaField); @@ -318,6 +326,8 @@ private ClassDefinition createNodeClass() { polymorphicTypeIdFieldDef = fieldDef; } else if (isInjectedName(schemaField)) { injectedNameFieldDef = fieldDef; + } else if (isInjectedValue(schemaField)) { + injectedValueField = schemaField; } } @@ -444,6 +454,10 @@ private ClassDefinition createNodeClass() { addInjectedNameFieldMethods(injectedNameFieldDef); } + if (injectedValueField != null) { + implementInjectedValueFieldNameMethod(injectedValueField); + } + if (polymorphicTypeIdFieldDef != null) { addIsPolymorphicMethod(); } @@ -1578,6 +1592,16 @@ private void addInjectedNameFieldMethods(FieldDefinition injectedNameFieldDef) { )).ret(); } + private void implementInjectedValueFieldNameMethod(Field injectedValueField) { + MethodDefinition method = innerNodeClassDef.declareMethod( + EnumSet.of(PUBLIC), + INJECTED_VALUE_FIELD_NAME_MTD_NAME, + type(String.class) + ); + + method.getBody().append(constantString(publicName(injectedValueField))).retObject(); + } + /** * Adds an override for the {@link InnerNode#isPolymorphic} method that returns {@code true}. */ @@ -1698,7 +1722,7 @@ private ClassDefinition createPolymorphicExtensionNodeClass( ); // Read only. - if (isPolymorphicId(schemaField) || isInjectedName(schemaField)) { + if (isReadOnly(schemaField)) { continue; } diff --git a/modules/configuration/src/main/java/org/apache/ignite/internal/configuration/hocon/HoconListConfigurationSource.java b/modules/configuration/src/main/java/org/apache/ignite/internal/configuration/hocon/HoconListConfigurationSource.java index bcae565b7ce..e1c1486e9f5 100644 --- a/modules/configuration/src/main/java/org/apache/ignite/internal/configuration/hocon/HoconListConfigurationSource.java +++ b/modules/configuration/src/main/java/org/apache/ignite/internal/configuration/hocon/HoconListConfigurationSource.java @@ -105,35 +105,39 @@ public void descend(ConstructableTreeNode node) { String syntheticKeyName = ((NamedListNode) node).syntheticKeyName(); - int idx = 0; - for (Iterator iterator = hoconCfgList.iterator(); iterator.hasNext(); idx++) { - ConfigValue next = iterator.next(); + for (int idx = 0; idx < hoconCfgList.size(); idx++) { + ConfigValue next = hoconCfgList.get(idx); if (next.valueType() != ConfigValueType.OBJECT) { - throw new IllegalArgumentException( - format( - "'%s' is expected to be a composite configuration node, not a single value", - formatArrayPath(path, idx) - ) - ); + throw new IllegalArgumentException(format( + "'%s' is expected to be a composite configuration node, not a single value", + formatArrayPath(path, idx) + )); } ConfigObject hoconCfg = (ConfigObject) next; - ConfigValue keyValue = hoconCfg.get(syntheticKeyName); + String key; - if (keyValue == null || keyValue.valueType() != ConfigValueType.STRING) { - throw new IllegalArgumentException( - format( - "'%s' configuration value is mandatory and must be a String", - formatArrayPath(path, idx) + KEY_SEPARATOR + syntheticKeyName - ) - ); - } + List path; - String key = (String) keyValue.unwrapped(); + ConfigValue keyValue = hoconCfg.get(syntheticKeyName); - List path = appendKey(this.path, key); + if (keyValue != null && keyValue.valueType() == ConfigValueType.STRING) { + // If the synthetic key is present, check that it has the correct type and use it as the key. + key = (String) keyValue.unwrapped(); + path = appendKey(this.path, key); + } else if (keyValue == null && hoconCfg.size() == 1) { + // If the synthetic key is not present explicitly, we need to handle the case when a configuration uses InjectedValue. + // This means that this object must only have one key. + key = hoconCfg.entrySet().iterator().next().getKey(); + path = this.path; + } else { + throw new IllegalArgumentException(format( + "'%s' configuration value is mandatory and must be a String", + formatArrayPath(this.path, idx) + KEY_SEPARATOR + syntheticKeyName + )); + } node.construct(key, new HoconObjectConfigurationSource(syntheticKeyName, path, hoconCfg), false); } diff --git a/modules/configuration/src/main/java/org/apache/ignite/internal/configuration/hocon/HoconObjectConfigurationSource.java b/modules/configuration/src/main/java/org/apache/ignite/internal/configuration/hocon/HoconObjectConfigurationSource.java index b6e97ec7858..b6a39978609 100644 --- a/modules/configuration/src/main/java/org/apache/ignite/internal/configuration/hocon/HoconObjectConfigurationSource.java +++ b/modules/configuration/src/main/java/org/apache/ignite/internal/configuration/hocon/HoconObjectConfigurationSource.java @@ -27,7 +27,6 @@ import com.typesafe.config.ConfigValue; import com.typesafe.config.ConfigValueType; import java.util.List; -import java.util.Map; import java.util.NoSuchElementException; import org.apache.ignite.configuration.ConfigurationWrongPolymorphicTypeIdException; import org.apache.ignite.internal.configuration.tree.ConfigurationSource; @@ -67,76 +66,81 @@ class HoconObjectConfigurationSource implements ConfigurationSource { this.hoconCfgObject = hoconCfgObject; } - /** {@inheritDoc} */ @Override public T unwrap(Class clazz) { throw wrongTypeException(clazz, path, -1); } - /** {@inheritDoc} */ @Override public void descend(ConstructableTreeNode node) { - for (Map.Entry entry : hoconCfgObject.entrySet()) { - String key = entry.getKey(); + String injectedValueFieldName = node.injectedValueFieldName(); - if (key.equals(ignoredKey)) { - continue; - } + if (injectedValueFieldName == null) { + hoconCfgObject.forEach((key, value) -> parseConfigEntry(key, value, node)); + } else { + assert hoconCfgObject.size() == 1; // User-friendly check must have been performed outside this method. - ConfigValue hoconCfgValue = entry.getValue(); + ConfigValue value = hoconCfgObject.values().iterator().next(); - try { - switch (hoconCfgValue.valueType()) { - case NULL: - node.construct(key, null, false); + parseConfigEntry(injectedValueFieldName, value, node); + } + } - break; + private void parseConfigEntry(String key, ConfigValue hoconCfgValue, ConstructableTreeNode node) { + if (key.equals(ignoredKey)) { + return; + } - case OBJECT: { - List path = appendKey(this.path, key); + try { + switch (hoconCfgValue.valueType()) { + case NULL: + node.construct(key, null, false); - node.construct( - key, - new HoconObjectConfigurationSource(null, path, (ConfigObject) hoconCfgValue), - false - ); + break; - break; - } + case OBJECT: { + List path = appendKey(this.path, key); - case LIST: { - List path = appendKey(this.path, key); + node.construct( + key, + new HoconObjectConfigurationSource(null, path, (ConfigObject) hoconCfgValue), + false + ); - node.construct(key, new HoconListConfigurationSource(path, (ConfigList) hoconCfgValue), false); + break; + } - break; - } + case LIST: { + List path = appendKey(this.path, key); - default: { - List path = appendKey(this.path, key); + node.construct(key, new HoconListConfigurationSource(path, (ConfigList) hoconCfgValue), false); - node.construct(key, new HoconPrimitiveConfigurationSource(path, hoconCfgValue), false); - } + break; } - } catch (NoSuchElementException e) { - if (path.isEmpty()) { - throw new IllegalArgumentException( - format("'%s' configuration root doesn't exist", key), e - ); - } else { - throw new IllegalArgumentException( - format("'%s' configuration doesn't have the '%s' sub-configuration", join(path), key), e - ); + + default: { + List path = appendKey(this.path, key); + + node.construct(key, new HoconPrimitiveConfigurationSource(path, hoconCfgValue), false); } - } catch (ConfigurationWrongPolymorphicTypeIdException e) { + } + } catch (NoSuchElementException e) { + if (path.isEmpty()) { throw new IllegalArgumentException( - "Polymorphic configuration type is not correct: " + e.getMessage() + format("'%s' configuration root doesn't exist", key), e + ); + } else { + throw new IllegalArgumentException( + format("'%s' configuration doesn't have the '%s' sub-configuration", join(path), key), e ); } + } catch (ConfigurationWrongPolymorphicTypeIdException e) { + throw new IllegalArgumentException( + "Polymorphic configuration type is not correct: " + e.getMessage() + ); } } - /** {@inheritDoc} */ @Override public @Nullable String polymorphicTypeId(String fieldName) { ConfigValue typeId = hoconCfgObject.get(fieldName); diff --git a/modules/configuration/src/main/java/org/apache/ignite/internal/configuration/hocon/HoconPrimitiveConfigurationSource.java b/modules/configuration/src/main/java/org/apache/ignite/internal/configuration/hocon/HoconPrimitiveConfigurationSource.java index 830b2567e77..4c50b42b27b 100644 --- a/modules/configuration/src/main/java/org/apache/ignite/internal/configuration/hocon/HoconPrimitiveConfigurationSource.java +++ b/modules/configuration/src/main/java/org/apache/ignite/internal/configuration/hocon/HoconPrimitiveConfigurationSource.java @@ -58,7 +58,6 @@ class HoconPrimitiveConfigurationSource implements ConfigurationSource { this.hoconCfgValue = hoconCfgValue; } - /** {@inheritDoc} */ @Override public T unwrap(Class clazz) { if (clazz.isArray()) { @@ -68,12 +67,17 @@ public T unwrap(Class clazz) { return unwrapPrimitive(hoconCfgValue, clazz, path, -1); } - /** {@inheritDoc} */ @Override public void descend(ConstructableTreeNode node) { - throw new IllegalArgumentException( - format("'%s' is expected to be a composite configuration node, not a single value", join(path)) - ); + String fieldName = node.injectedValueFieldName(); + + if (fieldName == null) { + throw new IllegalArgumentException( + format("'%s' is expected to be a composite configuration node, not a single value", join(path)) + ); + } + + node.construct(fieldName, this, false); } /** diff --git a/modules/configuration/src/main/java/org/apache/ignite/internal/configuration/tree/ConstructableTreeNode.java b/modules/configuration/src/main/java/org/apache/ignite/internal/configuration/tree/ConstructableTreeNode.java index a82d45fa469..edd2c18343f 100644 --- a/modules/configuration/src/main/java/org/apache/ignite/internal/configuration/tree/ConstructableTreeNode.java +++ b/modules/configuration/src/main/java/org/apache/ignite/internal/configuration/tree/ConstructableTreeNode.java @@ -18,6 +18,8 @@ package org.apache.ignite.internal.configuration.tree; import java.util.NoSuchElementException; +import org.apache.ignite.configuration.annotation.InjectedValue; +import org.jetbrains.annotations.Nullable; /** * Interface for filling the configuration node. @@ -48,4 +50,11 @@ public interface ConstructableTreeNode { * @return {@code true} if node became immutable, {@code false} if it has already been immutable before. */ boolean makeImmutable(); + + /** + * Returns the name of the field annotated with {@link InjectedValue} or {@code null} if no such field exists. + */ + default @Nullable String injectedValueFieldName() { + return null; + } } diff --git a/modules/configuration/src/main/java/org/apache/ignite/internal/configuration/tree/ConverterToMapVisitor.java b/modules/configuration/src/main/java/org/apache/ignite/internal/configuration/tree/ConverterToMapVisitor.java index 76a396e6f73..4e90427b4f0 100644 --- a/modules/configuration/src/main/java/org/apache/ignite/internal/configuration/tree/ConverterToMapVisitor.java +++ b/modules/configuration/src/main/java/org/apache/ignite/internal/configuration/tree/ConverterToMapVisitor.java @@ -17,6 +17,8 @@ package org.apache.ignite.internal.configuration.tree; +import static org.apache.ignite.internal.util.IgniteUtils.newHashMap; + import java.io.Serializable; import java.lang.reflect.Array; import java.lang.reflect.Field; @@ -82,7 +84,7 @@ public Object visitLeafNode(Field field, String key, Serializable val) { if (val instanceof Character || val instanceof UUID) { valObj = val.toString(); } else if (val != null && val.getClass().isArray()) { - valObj = toListOfObjects(field, val); + valObj = toListOfObjects(val); } else if (val instanceof String) { valObj = maskIfNeeded(field, (String) val); } @@ -92,7 +94,6 @@ public Object visitLeafNode(Field field, String key, Serializable val) { return valObj; } - /** {@inheritDoc} */ @Override public Object visitInnerNode(Field field, String key, InnerNode node) { if (skipEmptyValues && node == null) { @@ -107,33 +108,74 @@ public Object visitInnerNode(Field field, String key, InnerNode node) { deque.pop(); - addToParent(key, innerMap); + String injectedValueFieldName = node.injectedValueFieldName(); + + if (injectedValueFieldName != null) { + // If configuration contains an injected value, the rendered named list will be a map, where every injected value is represented + // as a separate key-value pair. + Object injectedValue = innerMap.get(injectedValueFieldName); + + addToParent(key, injectedValue); + + return injectedValue; + } else { + // Otherwise, the rendered named list will be a list of maps. + addToParent(key, innerMap); - return innerMap; + return innerMap; + } } - /** {@inheritDoc} */ @Override public Object visitNamedListNode(Field field, String key, NamedListNode node) { - if (skipEmptyValues && node.size() == 0) { + if (skipEmptyValues && node.isEmpty()) { return null; } - List list = new ArrayList<>(node.size()); + Object renderedList; + + boolean hasInjectedValues = !node.isEmpty() && getFirstNode(node).injectedValueFieldName() != null; + + // See the comment inside "visitInnerNode" why named lists are rendered differently for injected values. + if (hasInjectedValues) { + Map map = newHashMap(node.size()); + + deque.push(map); - deque.push(list); + for (String subkey : node.namedListKeys()) { + InnerNode innerNode = node.getInnerNode(subkey); - for (String subkey : node.namedListKeys()) { - node.getInnerNode(subkey).accept(field, subkey, this); + innerNode.accept(field, subkey, this); + } + + renderedList = map; + } else { + List list = new ArrayList<>(node.size()); + + deque.push(list); + + for (String subkey : node.namedListKeys()) { + InnerNode innerNode = node.getInnerNode(subkey); + + innerNode.accept(field, subkey, this); - ((Map) list.get(list.size() - 1)).put(node.syntheticKeyName(), subkey); + ((Map) list.get(list.size() - 1)).put(node.syntheticKeyName(), subkey); + } + + renderedList = list; } deque.pop(); - addToParent(key, list); + addToParent(key, renderedList); + + return renderedList; + } + + private static InnerNode getFirstNode(NamedListNode namedListNode) { + String firstKey = namedListNode.namedListKeys().get(0); - return list; + return namedListNode.getInnerNode(firstKey); } /** @@ -173,11 +215,10 @@ private void addToParent(String key, Object val) { /** * Converts array into a list of objects. Boxes array elements if they are primitive values. * - * @param field Field of the array * @param val Array of primitives or array of {@link String}s * @return List of objects corresponding to the passed array. */ - private List toListOfObjects(Field field, Serializable val) { + private static List toListOfObjects(Serializable val) { Stream stream = IntStream.range(0, Array.getLength(val)).mapToObj(i -> Array.get(val, i)); if (val.getClass().getComponentType() == char.class || val.getClass().getComponentType() == UUID.class) { diff --git a/modules/configuration/src/main/java/org/apache/ignite/internal/configuration/util/ConfigurationUtil.java b/modules/configuration/src/main/java/org/apache/ignite/internal/configuration/util/ConfigurationUtil.java index 7ff2b2eae5f..a0e1a28b967 100644 --- a/modules/configuration/src/main/java/org/apache/ignite/internal/configuration/util/ConfigurationUtil.java +++ b/modules/configuration/src/main/java/org/apache/ignite/internal/configuration/util/ConfigurationUtil.java @@ -56,6 +56,7 @@ import org.apache.ignite.configuration.annotation.ConfigurationExtension; import org.apache.ignite.configuration.annotation.ConfigurationRoot; import org.apache.ignite.configuration.annotation.InjectedName; +import org.apache.ignite.configuration.annotation.InjectedValue; import org.apache.ignite.configuration.annotation.InternalId; import org.apache.ignite.configuration.annotation.Name; import org.apache.ignite.configuration.annotation.NamedConfigValue; @@ -443,7 +444,7 @@ public static void checkConfigurationType(Collection> rootKeys, Co * @return {@code true} if field represents primitive configuration. */ public static boolean isValue(Field schemaField) { - return schemaField.isAnnotationPresent(Value.class); + return schemaField.isAnnotationPresent(Value.class) || schemaField.isAnnotationPresent(InjectedValue.class); } /** @@ -498,6 +499,12 @@ public static String syntheticKeyName(Field field) { public static boolean hasDefault(Field field) { assert isValue(field) : field; + InjectedValue injectedValue = field.getAnnotation(InjectedValue.class); + + if (injectedValue != null) { + return injectedValue.hasDefault(); + } + return field.getAnnotation(Value.class).hasDefault(); } @@ -1129,6 +1136,20 @@ public static boolean isInjectedName(Field schemaField) { return schemaField.isAnnotationPresent(InjectedName.class); } + /** + * Returns {@code true} if the given schema field contains the {@link InjectedValue} annotation. + */ + public static boolean isInjectedValue(Field schemaField) { + return schemaField.isAnnotationPresent(InjectedValue.class); + } + + /** + * Returns {@code true} if the given schema field is read-only. + */ + public static boolean isReadOnly(Field schemaField) { + return isPolymorphicId(schemaField) || isInjectedName(schemaField) || isInternalId(schemaField); + } + /** * Checks whether configuration schema field contains {@link Name}. * diff --git a/modules/configuration/src/test/java/org/apache/ignite/internal/configuration/InjectedValueConfigurationTest.java b/modules/configuration/src/test/java/org/apache/ignite/internal/configuration/InjectedValueConfigurationTest.java new file mode 100644 index 00000000000..3d929c30558 --- /dev/null +++ b/modules/configuration/src/test/java/org/apache/ignite/internal/configuration/InjectedValueConfigurationTest.java @@ -0,0 +1,194 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.configuration; + +import static org.apache.ignite.configuration.annotation.ConfigurationType.LOCAL; +import static org.apache.ignite.internal.configuration.hocon.HoconConverter.hoconSource; +import static org.apache.ignite.internal.testframework.matchers.CompletableFutureMatcher.willCompleteSuccessfully; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.is; +import static org.junit.jupiter.api.Assertions.assertEquals; + +import com.typesafe.config.ConfigFactory; +import com.typesafe.config.ConfigRenderOptions; +import com.typesafe.config.ConfigValue; +import java.util.Arrays; +import java.util.List; +import java.util.concurrent.CompletableFuture; +import java.util.stream.Collectors; +import java.util.stream.Stream; +import org.apache.ignite.configuration.RootKey; +import org.apache.ignite.configuration.annotation.Config; +import org.apache.ignite.configuration.annotation.ConfigurationRoot; +import org.apache.ignite.configuration.annotation.InjectedName; +import org.apache.ignite.configuration.annotation.InjectedValue; +import org.apache.ignite.configuration.annotation.NamedConfigValue; +import org.apache.ignite.internal.configuration.hocon.HoconConverter; +import org.apache.ignite.internal.configuration.storage.TestConfigurationStorage; +import org.apache.ignite.internal.configuration.validation.TestConfigurationValidator; +import org.apache.ignite.internal.manager.ComponentContext; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Nested; +import org.junit.jupiter.api.Test; + +/** + * Tests for schemas with {@link InjectedValue}s. + */ +public class InjectedValueConfigurationTest { + /** Root schema. */ + @ConfigurationRoot(rootName = "rootInjectedValue", type = LOCAL) + public static class HoconInjectedValueRootConfigurationSchema { + @NamedConfigValue + public HoconInjectedValueConfigurationSchema nestedNamed; + } + + /** Named list element schema. */ + @Config + public static class HoconInjectedValueConfigurationSchema { + @InjectedName + public String someName; + + @InjectedValue(hasDefault = true) + public String someValue = "default"; + } + + private ConfigurationRegistry registry; + + @BeforeEach + void setUp() { + List> roots = List.of( + HoconInjectedValueRootConfiguration.KEY + ); + + registry = new ConfigurationRegistry( + roots, + new TestConfigurationStorage(LOCAL), + new ConfigurationTreeGenerator(roots, List.of(), List.of()), + new TestConfigurationValidator() + ); + + assertThat(registry.startAsync(new ComponentContext()), willCompleteSuccessfully()); + } + + @AfterEach + void tearDown() { + assertThat(registry.stopAsync(new ComponentContext()), willCompleteSuccessfully()); + } + + @Nested + class HoconConverterTest { + @Test + void testEmpty() { + assertEquals("nestedNamed=[]", asHoconStr(List.of("rootInjectedValue"))); + } + + @Test + void testAssignValuesUsingObjectNotation() { + change("rootInjectedValue.nestedNamed = {foo: bar}"); + + assertEquals("nestedNamed{foo=bar}", asHoconStr(List.of("rootInjectedValue"))); + + change("rootInjectedValue.nestedNamed = {foo: bar, baz: quux}"); + + assertEquals("nestedNamed{baz=quux,foo=bar}", asHoconStr(List.of("rootInjectedValue"))); + + change("rootInjectedValue.nestedNamed = {baz: anotherQuux}"); + + assertEquals("nestedNamed{baz=anotherQuux,foo=bar}", asHoconStr(List.of("rootInjectedValue"))); + + change("rootInjectedValue.nestedNamed = {baz: null}"); + + assertEquals("nestedNamed{foo=bar}", asHoconStr(List.of("rootInjectedValue"))); + } + + @Test + void testAssignValuesUsingListNotation() { + change("rootInjectedValue.nestedNamed = [{foo=bar}]"); + + assertEquals("nestedNamed{foo=bar}", asHoconStr(List.of("rootInjectedValue"))); + + change("rootInjectedValue.nestedNamed = [{foo=bar},{baz=quux}]"); + + assertEquals("nestedNamed{baz=quux,foo=bar}", asHoconStr(List.of("rootInjectedValue"))); + + change("rootInjectedValue.nestedNamed = [{baz=anotherQuux}]"); + + assertEquals("nestedNamed{baz=anotherQuux,foo=bar}", asHoconStr(List.of("rootInjectedValue"))); + + // Removing a value does not work in this notation. + } + } + + @Nested + class JavaApiTest { + @Test + void testEmpty() { + HoconInjectedValueRootConfiguration cfg = registry.getConfiguration(HoconInjectedValueRootConfiguration.KEY); + + assertThat(cfg.nestedNamed().value().size(), is(0)); + } + + @Test + void testDefaults() { + HoconInjectedValueRootConfiguration cfg = registry.getConfiguration(HoconInjectedValueRootConfiguration.KEY); + + CompletableFuture changeFuture = cfg.change(rootChange -> rootChange + .changeNestedNamed(nestedChange -> nestedChange + .create("foo", valueChange -> {}))); + + assertThat(changeFuture, willCompleteSuccessfully()); + + assertThat(cfg.value().nestedNamed().get("foo").someValue(), is("default")); + assertThat(cfg.nestedNamed().value().get("foo").someValue(), is("default")); + assertThat(cfg.nestedNamed().get("foo").value().someValue(), is("default")); + assertThat(cfg.nestedNamed().get("foo").someValue().value(), is("default")); + } + + @Test + void testAssignValues() { + HoconInjectedValueRootConfiguration cfg = registry.getConfiguration(HoconInjectedValueRootConfiguration.KEY); + + CompletableFuture changeFuture = cfg.change(rootChange -> rootChange + .changeNestedNamed(nestedChange -> nestedChange + .create("foo", valueChange -> valueChange.changeSomeValue("bar")))); + + assertThat(changeFuture, willCompleteSuccessfully()); + + assertThat(cfg.value().nestedNamed().get("foo").someValue(), is("bar")); + assertThat(cfg.nestedNamed().value().get("foo").someValue(), is("bar")); + assertThat(cfg.nestedNamed().get("foo").value().someValue(), is("bar")); + assertThat(cfg.nestedNamed().get("foo").someValue().value(), is("bar")); + } + } + + private void change(String hocon) { + assertThat( + registry.change(hoconSource(ConfigFactory.parseString(hocon).root())), + willCompleteSuccessfully() + ); + } + + private String asHoconStr(List basePath, String... path) { + List fullPath = Stream.concat(basePath.stream(), Arrays.stream(path)).collect(Collectors.toList()); + + ConfigValue hoconCfg = HoconConverter.represent(registry.superRoot(), fullPath); + + return hoconCfg.render(ConfigRenderOptions.concise().setJson(false)); + } +} diff --git a/modules/configuration/src/test/java/org/apache/ignite/internal/configuration/hocon/HoconConverterTest.java b/modules/configuration/src/test/java/org/apache/ignite/internal/configuration/hocon/HoconConverterTest.java index f403cb9312c..7909d9cd439 100644 --- a/modules/configuration/src/test/java/org/apache/ignite/internal/configuration/hocon/HoconConverterTest.java +++ b/modules/configuration/src/test/java/org/apache/ignite/internal/configuration/hocon/HoconConverterTest.java @@ -731,7 +731,7 @@ void testInjectedName() throws Throwable { // Let's check that the NamedConfigValue#syntheticKeyName key will not work. assertThrowsIllegalArgException( () -> change("rootInjectedName.nestedNamed = [{superName = foo}]"), - "'rootInjectedName.nestedNamed[0].someName' configuration value is mandatory and must be a String" + "'rootInjectedName.nestedNamed' configuration doesn't have the 'superName' sub-configuration" ); } diff --git a/modules/distribution-zones/src/integrationTest/java/org/apache/ignite/internal/distributionzones/ItDistributionZonesFiltersTest.java b/modules/distribution-zones/src/integrationTest/java/org/apache/ignite/internal/distributionzones/ItDistributionZonesFiltersTest.java index 8bfd18470fa..14786209b36 100644 --- a/modules/distribution-zones/src/integrationTest/java/org/apache/ignite/internal/distributionzones/ItDistributionZonesFiltersTest.java +++ b/modules/distribution-zones/src/integrationTest/java/org/apache/ignite/internal/distributionzones/ItDistributionZonesFiltersTest.java @@ -63,7 +63,7 @@ public class ItDistributionZonesFiltersTest extends ClusterPerTestIntegrationTes private static final String TABLE_NAME = "table1"; @Language("HOCON") - private static final String NODE_ATTRIBUTES = "{region.attribute = US, storage.attribute = SSD}"; + private static final String NODE_ATTRIBUTES = "{region = US, storage = SSD}"; private static final String STORAGE_PROFILES = String.format("'%s, %s'", DEFAULT_ROCKSDB_PROFILE_NAME, DEFAULT_AIPERSIST_PROFILE_NAME); @@ -115,7 +115,7 @@ void testFilteredDataNodesPropagatedToStable() throws Exception { String filter = "$[?(@.region == \"US\" && @.storage == \"SSD\")]"; // This node do not pass the filter - @Language("HOCON") String firstNodeAttributes = "{region:{attribute:\"EU\"},storage:{attribute:\"SSD\"}}"; + @Language("HOCON") String firstNodeAttributes = "{region: EU, storage: SSD}"; Ignite node = startNode(1, createStartConfig(firstNodeAttributes, STORAGE_PROFILES_CONFIGS)); @@ -123,10 +123,10 @@ void testFilteredDataNodesPropagatedToStable() throws Exception { node.sql().execute(null, createTableSql()); - MetaStorageManager metaStorageManager = (MetaStorageManager) IgniteTestUtils + MetaStorageManager metaStorageManager = IgniteTestUtils .getFieldValue(node, IgniteImpl.class, "metaStorageMgr"); - TableManager tableManager = (TableManager) IgniteTestUtils.getFieldValue(node, IgniteImpl.class, "distributedTblMgr"); + TableManager tableManager = IgniteTestUtils.getFieldValue(node, IgniteImpl.class, "distributedTblMgr"); TableViewInternal table = (TableViewInternal) tableManager.table(TABLE_NAME); @@ -140,7 +140,7 @@ void testFilteredDataNodesPropagatedToStable() throws Exception { TIMEOUT_MILLIS ); - @Language("HOCON") String secondNodeAttributes = "{region:{attribute:\"US\"},storage:{attribute:\"SSD\"}}"; + @Language("HOCON") String secondNodeAttributes = "{region: US, storage: SSD}"; // This node pass the filter but storage profiles of a node do not match zone's storage profiles. // TODO: https://issues.apache.org/jira/browse/IGNITE-21387 recovery of this node is failing, @@ -210,7 +210,7 @@ void testAlteringFiltersPropagatedDataNodesToStableImmediately() throws Exceptio TIMEOUT_MILLIS ); - @Language("HOCON") String firstNodeAttributes = "{region:{attribute:\"US\"},storage:{attribute:\"SSD\"}}"; + @Language("HOCON") String firstNodeAttributes = "{region: US, storage: SSD}"; // This node pass the filter startNode(1, createStartConfig(firstNodeAttributes, STORAGE_PROFILES_CONFIGS)); @@ -265,7 +265,7 @@ void testEmptyDataNodesDoNotPropagatedToStableAfterAlteringFilter() throws Excep TIMEOUT_MILLIS ); - @Language("HOCON") String firstNodeAttributes = "{region:{attribute:\"US\"},storage:{attribute:\"SSD\"}}"; + @Language("HOCON") String firstNodeAttributes = "{region: US, storage: SSD}"; // This node pass the filter startNode(1, createStartConfig(firstNodeAttributes, STORAGE_PROFILES_CONFIGS)); @@ -306,7 +306,7 @@ void testFilteredEmptyDataNodesDoNotTriggerRebalance() throws Exception { Ignite node0 = unwrapIgniteImpl(node(0)); // This node passes the filter - @Language("HOCON") String firstNodeAttributes = "{region:{attribute:\"EU\"},storage:{attribute:\"HDD\"}}"; + @Language("HOCON") String firstNodeAttributes = "{region: EU, storage: HDD}"; Ignite node1 = startNode(1, createStartConfig(firstNodeAttributes, STORAGE_PROFILES_CONFIGS)); @@ -324,7 +324,7 @@ void testFilteredEmptyDataNodesDoNotTriggerRebalance() throws Exception { node1.sql().execute(null, createTableSql()); - TableManager tableManager = (TableManager) IgniteTestUtils.getFieldValue(node0, IgniteImpl.class, "distributedTblMgr"); + TableManager tableManager = IgniteTestUtils.getFieldValue(node0, IgniteImpl.class, "distributedTblMgr"); TableViewInternal table = (TableViewInternal) tableManager.table(TABLE_NAME); @@ -350,7 +350,7 @@ void testFilteredEmptyDataNodesDoNotTriggerRebalanceOnReplicaUpdate() throws Exc IgniteImpl node0 = unwrapIgniteImpl(node(0)); // This node passes the filter - @Language("HOCON") String firstNodeAttributes = "{region:{attribute:\"EU\"},storage:{attribute:\"HDD\"}}"; + @Language("HOCON") String firstNodeAttributes = "{region: EU, storage: HDD}"; startNode(1, createStartConfig(firstNodeAttributes, STORAGE_PROFILES_CONFIGS)); diff --git a/modules/distribution-zones/src/integrationTest/java/org/apache/ignite/internal/rebalance/ItRebalanceTriggersRecoveryTest.java b/modules/distribution-zones/src/integrationTest/java/org/apache/ignite/internal/rebalance/ItRebalanceTriggersRecoveryTest.java index ac954febcd8..65729316584 100644 --- a/modules/distribution-zones/src/integrationTest/java/org/apache/ignite/internal/rebalance/ItRebalanceTriggersRecoveryTest.java +++ b/modules/distribution-zones/src/integrationTest/java/org/apache/ignite/internal/rebalance/ItRebalanceTriggersRecoveryTest.java @@ -62,7 +62,7 @@ public class ItRebalanceTriggersRecoveryTest extends ClusterPerTestIntegrationTe + " },\n" + " clientConnector: { port:{} },\n" + " nodeAttributes: {\n" - + " nodeAttributes: {region: {attribute: \"US\"}, zone: {attribute: \"global\"}}\n" + + " nodeAttributes: {region: US, zone: global}\n" + " },\n" + " rest.port: {},\n" + " failureHandler.dumpThreadsOnFailure: false\n" @@ -77,7 +77,7 @@ public class ItRebalanceTriggersRecoveryTest extends ClusterPerTestIntegrationTe + " },\n" + " clientConnector: { port:{} },\n" + " nodeAttributes: {\n" - + " nodeAttributes: {zone: {attribute: \"global\"}}\n" + + " nodeAttributes: {zone: global}\n" + " },\n" + " rest.port: {},\n" + " failureHandler.dumpThreadsOnFailure: false\n" diff --git a/modules/distribution-zones/src/test/java/org/apache/ignite/internal/distributionzones/BaseDistributionZoneManagerTest.java b/modules/distribution-zones/src/test/java/org/apache/ignite/internal/distributionzones/BaseDistributionZoneManagerTest.java index 251e3bbd603..26909c6b2ac 100644 --- a/modules/distribution-zones/src/test/java/org/apache/ignite/internal/distributionzones/BaseDistributionZoneManagerTest.java +++ b/modules/distribution-zones/src/test/java/org/apache/ignite/internal/distributionzones/BaseDistributionZoneManagerTest.java @@ -89,9 +89,7 @@ public abstract class BaseDistributionZoneManagerTest extends BaseIgniteAbstract private final List components = new ArrayList<>(); - @InjectConfiguration("mock.properties = {" - + PARTITION_DISTRIBUTION_RESET_TIMEOUT + ".propertyValue = \"" + IMMEDIATE_TIMER_VALUE + "\", " - + "}") + @InjectConfiguration("mock.properties." + PARTITION_DISTRIBUTION_RESET_TIMEOUT + " = \"" + IMMEDIATE_TIMER_VALUE + "\"") SystemDistributedConfiguration systemDistributedConfiguration; @BeforeEach diff --git a/modules/distribution-zones/tech-notes/filters.md b/modules/distribution-zones/tech-notes/filters.md index 0cad552679f..22269fdf695 100644 --- a/modules/distribution-zones/tech-notes/filters.md +++ b/modules/distribution-zones/tech-notes/filters.md @@ -16,8 +16,8 @@ In the `ignite-config.conf` it looks like this: ``` nodeAttributes.nodeAttributes { - region.attribute = "US" - storage.attribute = "SSD" + region = "US" + storage = "SSD" } ``` @@ -27,12 +27,8 @@ or like this in a `ignite-config.json`: { "nodeAttributes":{ "nodeAttributes":{ - "region":{ - "attribute":"US" - }, - "storage":{ - "attribute":"SSD" - } + "region": "US", + "storage": "SSD" } } } @@ -65,6 +61,3 @@ To check all capabilities of JSONPath, see https://github.com/json-path/JsonPath Note that as a default value for filter '$..*' filter is used, meaning that all nodes match the filter. Also it is important, that if there are no specified attributes for a node, it means that a node match only the default filter. - - - diff --git a/modules/metastorage/src/integrationTest/java/org/apache/ignite/internal/metastorage/TestMetasStorageUtils.java b/modules/metastorage/src/integrationTest/java/org/apache/ignite/internal/metastorage/TestMetasStorageUtils.java index 67deee87073..40a4482d30b 100644 --- a/modules/metastorage/src/integrationTest/java/org/apache/ignite/internal/metastorage/TestMetasStorageUtils.java +++ b/modules/metastorage/src/integrationTest/java/org/apache/ignite/internal/metastorage/TestMetasStorageUtils.java @@ -81,8 +81,8 @@ public static boolean equals(Entry act, Entry exp) { public static String createClusterConfigWithCompactionProperties(long interval, long dataAvailabilityTime) { return String.format( "ignite.system.properties: {" - + "%s.propertyValue= \"%s\", " - + "%s.propertyValue= \"%s\"" + + "%s = \"%s\", " + + "%s = \"%s\"" + "}", INTERVAL_SYSTEM_PROPERTY_NAME, interval, DATA_AVAILABILITY_TIME_SYSTEM_PROPERTY_NAME, dataAvailabilityTime ); diff --git a/modules/metastorage/src/test/java/org/apache/ignite/internal/metastorage/impl/MetaStorageCompactionTriggerConfigurationTest.java b/modules/metastorage/src/test/java/org/apache/ignite/internal/metastorage/impl/MetaStorageCompactionTriggerConfigurationTest.java index 296fed8b936..17364d6ee4f 100644 --- a/modules/metastorage/src/test/java/org/apache/ignite/internal/metastorage/impl/MetaStorageCompactionTriggerConfigurationTest.java +++ b/modules/metastorage/src/test/java/org/apache/ignite/internal/metastorage/impl/MetaStorageCompactionTriggerConfigurationTest.java @@ -52,8 +52,8 @@ void testEmptySystemProperties(@InjectConfiguration SystemDistributedConfigurati @Test void testValidSystemPropertiesOnStart( @InjectConfiguration("mock.properties = {" - + INTERVAL_SYSTEM_PROPERTY_NAME + ".propertyValue = \"100\", " - + DATA_AVAILABILITY_TIME_SYSTEM_PROPERTY_NAME + ".propertyValue = \"500\"" + + INTERVAL_SYSTEM_PROPERTY_NAME + " = \"100\", " + + DATA_AVAILABILITY_TIME_SYSTEM_PROPERTY_NAME + " = \"500\"" + "}") SystemDistributedConfiguration systemConfig ) { diff --git a/modules/partition-replicator/src/integrationTest/java/org/apache/ignite/internal/partition/replicator/ItReplicaLifecycleTest.java b/modules/partition-replicator/src/integrationTest/java/org/apache/ignite/internal/partition/replicator/ItReplicaLifecycleTest.java index 749194de65a..6280a5e0f6c 100644 --- a/modules/partition-replicator/src/integrationTest/java/org/apache/ignite/internal/partition/replicator/ItReplicaLifecycleTest.java +++ b/modules/partition-replicator/src/integrationTest/java/org/apache/ignite/internal/partition/replicator/ItReplicaLifecycleTest.java @@ -256,13 +256,13 @@ public class ItReplicaLifecycleTest extends BaseIgniteAbstractTest { @InjectConfiguration private static SystemLocalConfiguration systemConfiguration; - @InjectConfiguration("mock.nodeAttributes: {region.attribute = US, storage.attribute = SSD}") + @InjectConfiguration("mock.nodeAttributes: {region = US, storage = SSD}") private static NodeAttributesConfiguration nodeAttributes1; - @InjectConfiguration("mock.nodeAttributes: {region.attribute = EU, storage.attribute = SSD}") + @InjectConfiguration("mock.nodeAttributes: {region = EU, storage = SSD}") private static NodeAttributesConfiguration nodeAttributes2; - @InjectConfiguration("mock.nodeAttributes: {region.attribute = UK, storage.attribute = SSD}") + @InjectConfiguration("mock.nodeAttributes: {region = UK, storage = SSD}") private static NodeAttributesConfiguration nodeAttributes3; @InjectConfiguration diff --git a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/ItIgniteNodeRestartTest.java b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/ItIgniteNodeRestartTest.java index e9efcc2d8c9..3e86b870d6a 100644 --- a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/ItIgniteNodeRestartTest.java +++ b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/ItIgniteNodeRestartTest.java @@ -1028,8 +1028,8 @@ public void changeNodeAttributesConfigurationOnStartTest() { stopNode(0); String newAttributesCfg = "{\n" - + " region.attribute = \"US\"\n" - + " storage.attribute = \"SSD\"\n" + + " region = US\n" + + " storage = SSD\n" + "}"; Map newAttributesMap = Map.of("region", "US", "storage", "SSD"); diff --git a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/ItReplicaStateManagerTest.java b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/ItReplicaStateManagerTest.java index 6f2cd40ea40..615a03b0bce 100644 --- a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/ItReplicaStateManagerTest.java +++ b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/ItReplicaStateManagerTest.java @@ -52,9 +52,9 @@ */ public class ItReplicaStateManagerTest extends BaseIgniteRestartTest { private static final String[] ATTRIBUTES = { - "{region:{attribute:\"REG0\"}}", - "{region:{attribute:\"REG1\"}}", - "{region:{attribute:\"REG2\"}}" + "{ region: REG0 }", + "{ region: REG1 }", + "{ region: REG2 }" }; private static final String ZONE_NAME = "TEST_ZONE"; diff --git a/modules/sql-engine/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItUnstableTopologyTest.java b/modules/sql-engine/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItUnstableTopologyTest.java index 6c703266b78..3c716423b98 100644 --- a/modules/sql-engine/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItUnstableTopologyTest.java +++ b/modules/sql-engine/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItUnstableTopologyTest.java @@ -37,7 +37,7 @@ public class ItUnstableTopologyTest extends BaseSqlIntegrationTest { + " },\n" + " clientConnector: { port:{} },\n" + " nodeAttributes: {\n" - + " nodeAttributes: {role: {attribute: \"data\"}}\n" + + " nodeAttributes: { role: data }\n" + " },\n" + " rest.port: {},\n" + " failureHandler.dumpThreadsOnFailure: false\n" diff --git a/modules/table/src/integrationTest/java/org/apache/ignite/internal/table/distributed/disaster/ItHighAvailablePartitionsRecoveryByFilterUpdateTest.java b/modules/table/src/integrationTest/java/org/apache/ignite/internal/table/distributed/disaster/ItHighAvailablePartitionsRecoveryByFilterUpdateTest.java index 08dc440b6a8..7456704f5f9 100644 --- a/modules/table/src/integrationTest/java/org/apache/ignite/internal/table/distributed/disaster/ItHighAvailablePartitionsRecoveryByFilterUpdateTest.java +++ b/modules/table/src/integrationTest/java/org/apache/ignite/internal/table/distributed/disaster/ItHighAvailablePartitionsRecoveryByFilterUpdateTest.java @@ -24,11 +24,11 @@ /** Test suite for the cases with a recovery of the group replication factor after reset by zone filter update. */ public class ItHighAvailablePartitionsRecoveryByFilterUpdateTest extends AbstractHighAvailablePartitionsRecoveryTest { - private static final String GLOBAL_EU_NODES_CONFIG = nodeConfig("{region.attribute = EU, zone.attribute = global}"); + private static final String GLOBAL_EU_NODES_CONFIG = nodeConfig("{region = EU, zone = global}"); - private static final String EU_ONLY_NODES_CONFIG = nodeConfig("{region.attribute = EU}"); + private static final String EU_ONLY_NODES_CONFIG = nodeConfig("{region = EU}"); - private static final String GLOBAL_NODES_CONFIG = nodeConfig("{zone.attribute = global}"); + private static final String GLOBAL_NODES_CONFIG = nodeConfig("{zone = global}"); @Override protected int initialNodes() { From afa41665a57b08c242f034aaa0ea7b7b68c4178f Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sat, 11 Jan 2025 16:30:21 +0200 Subject: [PATCH 011/113] Bump org.rocksdb:rocksdbjni from 9.7.3 to 9.7.4 (#5025) Bumps [org.rocksdb:rocksdbjni](https://github.com/facebook/rocksdb) from 9.7.3 to 9.7.4. - [Release notes](https://github.com/facebook/rocksdb/releases) - [Changelog](https://github.com/facebook/rocksdb/blob/v9.7.4/HISTORY.md) - [Commits](https://github.com/facebook/rocksdb/compare/v9.7.3...v9.7.4) --- updated-dependencies: - dependency-name: org.rocksdb:rocksdbjni dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: Aleksandr Polovtsev Co-authored-by: Aleksandr Polovtsev --- gradle/libs.versions.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index ddee42b0d44..7116ce1d4ac 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -61,7 +61,7 @@ value = "2.10.1" janino = "3.1.12" jsonpath = "2.9.0" javassist = "3.30.2-GA" -rocksdb = "9.7.3" +rocksdb = "9.7.4" disruptor = "4.0.0" metrics = "4.2.29" jctools = "4.0.5" From 4423d07d72c60d420598931c462b612ff7aa6fec Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sat, 11 Jan 2025 16:30:43 +0200 Subject: [PATCH 012/113] Bump com.github.spotbugs from 6.0.27 to 6.1.0 (#5024) Bumps com.github.spotbugs from 6.0.27 to 6.1.0. --- updated-dependencies: - dependency-name: com.github.spotbugs dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: Aleksandr Polovtsev Co-authored-by: Aleksandr Polovtsev --- gradle/libs.versions.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index 7116ce1d4ac..34bda13553f 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -104,7 +104,7 @@ checksum = "org.gradle.crypto.checksum:1.4.0" setupbuilder = "de.inetsoftware.setupbuilder:8.4.21" aggregateJavadoc = "io.freefair.aggregate-javadoc:6.6.3" ideaext = "org.jetbrains.gradle.plugin.idea-ext:1.1.9" -spotbugs = "com.github.spotbugs:6.0.27" +spotbugs = "com.github.spotbugs:6.1.0" [libraries] From 4dcc0b2935bf0cb60e6926b484c236cfeb8557c9 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sat, 11 Jan 2025 16:31:06 +0200 Subject: [PATCH 013/113] Bump org.springframework.data:spring-data-jdbc from 3.4.0 to 3.4.1 (#5003) Bumps org.springframework.data:spring-data-jdbc from 3.4.0 to 3.4.1. --- updated-dependencies: - dependency-name: org.springframework.data:spring-data-jdbc dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: Aleksandr Polovtsev Co-authored-by: Aleksandr Polovtsev --- gradle/libs.versions.toml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index 34bda13553f..be391c68d35 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -85,7 +85,7 @@ tree-sitter-sql = "gh-pages-a" tree-sitter-hocon = "master-a" otel = "1.45.0" spring-boot = "3.4.1" -spring-data = "3.4.0" +spring-data = "3.4.1" #Tools pmdTool = "6.55.0" From bfdfd239d32c83a9893248f68aaecbce9c77b587 Mon Sep 17 00:00:00 2001 From: Vladislav Pyatkov Date: Sat, 11 Jan 2025 19:14:34 +0300 Subject: [PATCH 014/113] IGNITE-24062 Use throttled logger for heap lock manager (#5015) --- .../logger/IgniteThrottledLogger.java | 13 +++++--- .../logger/IgniteThrottledLoggerImpl.java | 8 +++-- .../internal/tx/impl/HeapLockManager.java | 30 +++++++------------ 3 files changed, 25 insertions(+), 26 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/logger/IgniteThrottledLogger.java b/modules/core/src/main/java/org/apache/ignite/internal/logger/IgniteThrottledLogger.java index a97b47261a9..8aee2736725 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/logger/IgniteThrottledLogger.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/logger/IgniteThrottledLogger.java @@ -25,12 +25,17 @@ /** * {@link IgniteLogger} throttle. * - *

Messages are logged only if they were not logged for the last {@link #THROTTLE_TIMEOUT_MILLIS} milliseconds. Note that not only error - * messages are checked for duplicates, but also exception classes if present.

+ *

Messages are logged only if they were not logged recently. The interval of message appears is + * {@link this#DEFAULT_LOG_THROTTLE_INTERVAL_MS} by default or can be configured through the JVM property + * {@link this#LOG_THROTTLE_INTERVAL_MS}. Note that not only error messages are checked for duplicates, but also exception classes if + * present.

*/ public interface IgniteThrottledLogger extends IgniteLogger { - /** Throttle timeout in milliseconds (value is 5 min). */ - long THROTTLE_TIMEOUT_MILLIS = TimeUnit.MINUTES.toMillis(5); + /** JVM property to configure a throttle interval. */ + String LOG_THROTTLE_INTERVAL_MS = "IGNITE_LOG_THROTTLE_INTERVAL_MS"; + + /** Default log throttle interval in milliseconds (value is 5 min). */ + long DEFAULT_LOG_THROTTLE_INTERVAL_MS = TimeUnit.MINUTES.toMillis(5); /** * Logs a message on {@link Level#INFO} level composed from args with given format. diff --git a/modules/core/src/main/java/org/apache/ignite/internal/logger/IgniteThrottledLoggerImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/logger/IgniteThrottledLoggerImpl.java index 6d5a8d587ed..50281092de9 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/logger/IgniteThrottledLoggerImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/logger/IgniteThrottledLoggerImpl.java @@ -26,10 +26,14 @@ import java.util.concurrent.TimeUnit; import java.util.function.Supplier; import org.apache.ignite.internal.lang.IgniteStringFormatter; +import org.apache.ignite.internal.lang.IgniteSystemProperties; import org.apache.ignite.internal.util.FastTimestamps; import org.jetbrains.annotations.Nullable; class IgniteThrottledLoggerImpl implements IgniteThrottledLogger { + /** Throttle interval in milliseconds (value is 5 min). */ + private final long throttleIntervalMs = IgniteSystemProperties.getLong(LOG_THROTTLE_INTERVAL_MS, DEFAULT_LOG_THROTTLE_INTERVAL_MS); + /** Logger delegate. */ private final System.Logger delegate; @@ -41,7 +45,7 @@ class IgniteThrottledLoggerImpl implements IgniteThrottledLogger { messagesMap = Caffeine.newBuilder() .executor(executor) - .expireAfterWrite(THROTTLE_TIMEOUT_MILLIS, TimeUnit.MILLISECONDS) + .expireAfterWrite(throttleIntervalMs, TimeUnit.MILLISECONDS) .build() .asMap(); } @@ -285,7 +289,7 @@ private void logInternal( long curTs = FastTimestamps.coarseCurrentTimeMillis(); - if (loggedTs == null || curTs - loggedTs >= THROTTLE_TIMEOUT_MILLIS) { + if (loggedTs == null || curTs - loggedTs >= throttleIntervalMs) { if (replace(msgKey, loggedTs, curTs)) { if (throwable == null) { delegate.log(level, message); diff --git a/modules/transactions/src/main/java/org/apache/ignite/internal/tx/impl/HeapLockManager.java b/modules/transactions/src/main/java/org/apache/ignite/internal/tx/impl/HeapLockManager.java index bf0e7709b82..84a52c8ea5d 100644 --- a/modules/transactions/src/main/java/org/apache/ignite/internal/tx/impl/HeapLockManager.java +++ b/modules/transactions/src/main/java/org/apache/ignite/internal/tx/impl/HeapLockManager.java @@ -22,7 +22,6 @@ import static java.util.concurrent.CompletableFuture.failedFuture; import static org.apache.ignite.internal.tx.event.LockEvent.LOCK_CONFLICT; import static org.apache.ignite.internal.util.CompletableFutures.nullCompletedFuture; -import static org.apache.ignite.internal.util.FastTimestamps.coarseCurrentTimeMillis; import static org.apache.ignite.lang.ErrorGroups.Transactions.ACQUIRE_LOCK_ERR; import static org.apache.ignite.lang.ErrorGroups.Transactions.ACQUIRE_LOCK_TIMEOUT_ERR; @@ -44,11 +43,10 @@ import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.Executor; +import java.util.concurrent.ForkJoinPool; import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicLong; import org.apache.ignite.internal.event.AbstractEventProducer; import org.apache.ignite.internal.lang.IgniteBiTuple; -import org.apache.ignite.internal.lang.IgniteSystemProperties; import org.apache.ignite.internal.logger.IgniteLogger; import org.apache.ignite.internal.logger.Loggers; import org.apache.ignite.internal.tostring.IgniteToStringExclude; @@ -80,7 +78,12 @@ *

Additionally limits the lock map size. */ public class HeapLockManager extends AbstractEventProducer implements LockManager { - private static final IgniteLogger LOG = Loggers.forClass(HeapLockManager.class); + /** Throttled logger. */ + private static final IgniteLogger THROTTLED_LOG = Loggers.toThrottledLogger( + Loggers.forClass(HeapLockManager.class), + // TODO: IGNITE-24181 Get rid of Common thread pool. + ForkJoinPool.commonPool() + ); /** * Table size. TODO make it configurable IGNITE-20694 @@ -92,9 +95,6 @@ public class HeapLockManager extends AbstractEventProducer coarseMap = new ConcurrentHashMap<>(); - /** - * The last timestamp of the printing warning about exhausted lock slots. It is {@code 0} if the warning has never been printed. - * The timestamp is updated periodically {@link this#warningPrintInterval}. - */ - private final AtomicLong lastTsPintWarn = new AtomicLong(); - /** * Creates an instance of {@link HeapLockManager} with a few slots eligible for tests which don't stress the lock manager too much. * Such a small instance is started way faster than a full-blown production ready instance with a lot of slots. @@ -309,13 +303,9 @@ private LockState lockState(LockKey key) { res[0] = slots[index]; assert !res[0].markedForRemove; - long currentTs = coarseCurrentTimeMillis(); - - long previousTs = lastTsPintWarn.get(); - - if ((currentTs - previousTs) > warningPrintInterval && lastTsPintWarn.compareAndSet(previousTs, currentTs)) { - LOG.warn("Log manager runs out of slots. So the lock state starts to share, and conflicts may appear frequently."); - } + THROTTLED_LOG.warn( + "Log manager runs out of slots. So the lock state starts to share, and conflicts may appear frequently." + ); } else { v.markedForRemove = false; v.key = k; From a57ee5f9e6e4407d36f4cb2f32e123b4148aa562 Mon Sep 17 00:00:00 2001 From: Pavel Tupitsyn Date: Mon, 13 Jan 2025 11:29:47 +0200 Subject: [PATCH 015/113] IGNITE-24059 Reuse compute serialization in client and embedded APIs (#5017) * Extract common logic to `SharedComputeUtils`, reuse in client and embedded APIs * Always use `ComputeJobDataHolder` to pass argument and result data around internally * Before that we used internal message serialization for args and results in some cases, resulting in inconsistent behavior * Now we have a clear boundary where the serialization happens * Local execution also performs serialization - to be fixed in IGNITE-24183 --- .../client/proto/ClientComputeJobPacker.java | 110 +------- .../proto/ClientComputeJobUnpacker.java | 103 +------ .../compute/ComputeJobDataHolder.java | 0 .../internal/compute/ComputeJobDataType.java | 0 .../ignite/internal/compute/JobStateImpl.java | 0 .../internal/compute/JobTaskStatusMapper.java | 0 .../compute/PojoConversionException.java | 0 .../internal/compute/PojoConverter.java | 0 .../internal/compute/SharedComputeUtils.java | 261 ++++++++++++++++++ .../internal/compute/TaskStateImpl.java | 0 .../ClientComputeJobPackerUnpackerTest.java | 10 +- .../ClientComputeExecuteColocatedRequest.java | 7 +- .../compute/ClientComputeExecuteRequest.java | 11 +- .../ignite/client/fakes/FakeCompute.java | 77 ++++-- modules/compute/build.gradle | 1 + .../internal/compute/ItComputeBaseTest.java | 38 +-- .../internal/compute/ItComputeTestClient.java | 31 --- .../compute/ItComputeTestEmbedded.java | 62 +++-- .../compute/ItComputeTestStandalone.java | 6 +- .../{ConcatJob.java => ToStringJob.java} | 12 +- .../internal/compute/ComputeComponent.java | 35 +-- .../compute/ComputeComponentImpl.java | 40 +-- .../internal/compute/ComputeJobFailover.java | 20 +- .../ignite/internal/compute/ComputeUtils.java | 213 +------------- .../compute/DelegatingJobExecution.java | 30 +- .../internal/compute/ExecutionManager.java | 16 +- .../internal/compute/IgniteComputeImpl.java | 49 ++-- .../compute/IgniteComputeInternal.java | 10 +- .../ignite/internal/compute/JobStarter.java | 6 +- .../compute/RemoteExecutionContext.java | 22 +- .../ResultUnmarshallingJobExecution.java | 16 +- .../compute/executor/ComputeExecutor.java | 6 +- .../compute/executor/ComputeExecutorImpl.java | 31 ++- .../executor/JobExecutionInternal.java | 7 +- .../compute/message/ExecuteRequest.java | 3 +- .../compute/message/JobResultResponse.java | 3 +- .../compute/messaging/ComputeMessaging.java | 18 +- .../compute/ComputeComponentImplTest.java | 60 ++-- .../compute/IgniteComputeImplTest.java | 30 +- .../compute/executor/ComputeExecutorTest.java | 51 ++-- .../ItThinClientComputeMarshallingTest.java | 4 +- ...ClientComputeTypeCheckMarshallingTest.java | 10 +- ...tThinClientPojoComputeMarshallingTest.java | 6 +- .../compute/ItEmbeddedMarshallingTest.java | 13 +- 44 files changed, 654 insertions(+), 774 deletions(-) rename modules/{core => client-common}/src/main/java/org/apache/ignite/internal/compute/ComputeJobDataHolder.java (100%) rename modules/{core => client-common}/src/main/java/org/apache/ignite/internal/compute/ComputeJobDataType.java (100%) rename modules/{core => client-common}/src/main/java/org/apache/ignite/internal/compute/JobStateImpl.java (100%) rename modules/{core => client-common}/src/main/java/org/apache/ignite/internal/compute/JobTaskStatusMapper.java (100%) rename modules/{core => client-common}/src/main/java/org/apache/ignite/internal/compute/PojoConversionException.java (100%) rename modules/{core => client-common}/src/main/java/org/apache/ignite/internal/compute/PojoConverter.java (100%) create mode 100644 modules/client-common/src/main/java/org/apache/ignite/internal/compute/SharedComputeUtils.java rename modules/{core => client-common}/src/main/java/org/apache/ignite/internal/compute/TaskStateImpl.java (100%) rename modules/compute/src/jobs/java/org/apache/ignite/internal/compute/{ConcatJob.java => ToStringJob.java} (75%) diff --git a/modules/client-common/src/main/java/org/apache/ignite/internal/client/proto/ClientComputeJobPacker.java b/modules/client-common/src/main/java/org/apache/ignite/internal/client/proto/ClientComputeJobPacker.java index 4ea19494f54..d877c808999 100644 --- a/modules/client-common/src/main/java/org/apache/ignite/internal/client/proto/ClientComputeJobPacker.java +++ b/modules/client-common/src/main/java/org/apache/ignite/internal/client/proto/ClientComputeJobPacker.java @@ -17,34 +17,13 @@ package org.apache.ignite.internal.client.proto; -import static org.apache.ignite.internal.compute.ComputeJobDataType.MARSHALLED_CUSTOM; -import static org.apache.ignite.internal.compute.ComputeJobDataType.NATIVE; -import static org.apache.ignite.internal.compute.ComputeJobDataType.POJO; -import static org.apache.ignite.internal.compute.ComputeJobDataType.TUPLE; -import static org.apache.ignite.internal.compute.ComputeJobDataType.TUPLE_COLLECTION; -import static org.apache.ignite.internal.compute.PojoConverter.toTuple; - -import java.nio.ByteBuffer; -import java.util.Arrays; -import java.util.Collection; -import java.util.Set; -import java.util.stream.Collectors; -import org.apache.ignite.internal.binarytuple.BinaryTupleBuilder; -import org.apache.ignite.internal.binarytuple.inlineschema.TupleWithSchemaMarshalling; import org.apache.ignite.internal.compute.ComputeJobDataHolder; -import org.apache.ignite.internal.compute.PojoConversionException; +import org.apache.ignite.internal.compute.SharedComputeUtils; import org.apache.ignite.marshalling.Marshaller; -import org.apache.ignite.marshalling.MarshallingException; -import org.apache.ignite.sql.ColumnType; -import org.apache.ignite.table.Tuple; import org.jetbrains.annotations.Nullable; /** Packs job arguments and results. */ public final class ClientComputeJobPacker { - private static final Set> NATIVE_TYPES = Arrays.stream(ColumnType.values()) - .map(ColumnType::javaClass) - .collect(Collectors.toUnmodifiableSet()); - /** * Packs compute job argument. If the marshaller is provided, it will be used to marshal the argument. If the marshaller is not provided * and the argument is a native column type or a tuple, it will be packed accordingly. @@ -68,92 +47,21 @@ public static void packJobArgument(@Nullable T arg, @Nullable Marshaller Result type. */ public static void packJobResult(@Nullable T res, @Nullable Marshaller marshaller, ClientMessagePacker packer) { - if (res instanceof ComputeJobDataHolder) { - ComputeJobDataHolder resultDataHolder = (ComputeJobDataHolder) res; - packer.packInt(resultDataHolder.type().id()); - packer.packBinary(resultDataHolder.data()); - } else { - pack(res, marshaller, packer); - } + pack(res, marshaller, packer); } /** Packs object in the format: | typeId | value |. */ private static void pack(@Nullable T obj, @Nullable Marshaller marshaller, ClientMessagePacker packer) { - if (obj == null) { - packer.packNil(); - return; - } - - if (marshaller != null) { - packer.packInt(MARSHALLED_CUSTOM.id()); - byte[] marshalled = marshaller.marshal(obj); - - if (marshalled == null) { - packer.packNil(); - return; - } - - packer.packBinary(marshalled); - return; - } - - if (obj instanceof Tuple) { - packer.packInt(TUPLE.id()); - - packTuple((Tuple) obj, packer); - return; - } - - if (obj instanceof Collection) { - Collection col = (Collection) obj; - - packer.packInt(TUPLE_COLLECTION.id()); + ComputeJobDataHolder holder = obj instanceof ComputeJobDataHolder + ? (ComputeJobDataHolder) obj + : SharedComputeUtils.marshalArgOrResult(obj, marshaller); - // Pack entire collection into a single binary blob. - BinaryTupleBuilder tupleBuilder = new BinaryTupleBuilder(col.size()); - - for (Object el : col) { - if (el == null) { - tupleBuilder.appendNull(); - continue; - } - - if (!(el instanceof Tuple)) { - throw new MarshallingException("Can't pack collection: expected Tuple, but got " + el.getClass(), null); - } - - tupleBuilder.appendBytes(TupleWithSchemaMarshalling.marshal((Tuple) el)); - } - - ByteBuffer binTupleBytes = tupleBuilder.build(); - packer.packBinaryHeader(Integer.BYTES + binTupleBytes.remaining()); - packer.writeIntRawLittleEndian(col.size()); - packer.writePayload(binTupleBytes); - - return; - } - - if (isNativeType(obj.getClass())) { - packer.packInt(NATIVE.id()); - - packer.packObjectAsBinaryTuple(obj); + if (holder == null) { + packer.packNil(); return; } - try { - packer.packInt(POJO.id()); - - packTuple(toTuple(obj), packer); - } catch (PojoConversionException e) { - throw new MarshallingException("Can't pack object: " + obj, e); - } - } - - private static boolean isNativeType(Class clazz) { - return NATIVE_TYPES.contains(clazz); - } - - private static void packTuple(Tuple tuple, ClientMessagePacker packer) { - packer.packBinary(TupleWithSchemaMarshalling.marshal(tuple)); + packer.packInt(holder.type().id()); + packer.packBinary(holder.data()); } } diff --git a/modules/client-common/src/main/java/org/apache/ignite/internal/client/proto/ClientComputeJobUnpacker.java b/modules/client-common/src/main/java/org/apache/ignite/internal/client/proto/ClientComputeJobUnpacker.java index a7c78564413..ed2c2ef7322 100644 --- a/modules/client-common/src/main/java/org/apache/ignite/internal/client/proto/ClientComputeJobUnpacker.java +++ b/modules/client-common/src/main/java/org/apache/ignite/internal/client/proto/ClientComputeJobUnpacker.java @@ -17,22 +17,11 @@ package org.apache.ignite.internal.client.proto; -import static org.apache.ignite.internal.compute.PojoConverter.fromTuple; -import static org.apache.ignite.marshalling.Marshaller.tryUnmarshalOrCast; - -import java.lang.reflect.InvocationTargetException; -import java.nio.ByteBuffer; -import java.nio.ByteOrder; -import java.util.ArrayList; -import java.util.List; -import org.apache.ignite.internal.binarytuple.BinaryTupleReader; -import org.apache.ignite.internal.binarytuple.inlineschema.TupleWithSchemaMarshalling; import org.apache.ignite.internal.compute.ComputeJobDataHolder; import org.apache.ignite.internal.compute.ComputeJobDataType; -import org.apache.ignite.internal.compute.PojoConversionException; +import org.apache.ignite.internal.compute.SharedComputeUtils; import org.apache.ignite.marshalling.Marshaller; import org.apache.ignite.marshalling.UnmarshallingException; -import org.apache.ignite.table.Tuple; import org.jetbrains.annotations.Nullable; /** Unpacks job results. */ @@ -52,97 +41,13 @@ public final class ClientComputeJobUnpacker { @Nullable Marshaller marshaller, @Nullable Class resultClass ) { - if (unpacker.tryUnpackNil()) { - return null; - } - - // Underlying byte array expected to be in the following format: | typeId | value |. - int typeId = unpacker.unpackInt(); - ComputeJobDataType type = ComputeJobDataType.fromId(typeId); - if (type == null) { - throw new UnmarshallingException("Unsupported compute job type id: " + typeId); - } - - switch (type) { - case NATIVE: - if (marshaller != null) { - throw new UnmarshallingException( - "Can not unpack object because the marshaller is provided but the object was packed without marshaller." - ); - } - - return unpacker.unpackObjectFromBinaryTuple(); - - case TUPLE: - return TupleWithSchemaMarshalling.unmarshal(unpacker.readBinary()); + ComputeJobDataHolder holder = unpackJobArgumentWithoutMarshaller(unpacker); - case MARSHALLED_CUSTOM: - if (marshaller == null) { - throw new UnmarshallingException( - "Can not unpack object because the marshaller is not provided but the object was packed with marshaller." - ); - } - return tryUnmarshalOrCast(marshaller, unpacker.readBinary()); - - case POJO: - if (resultClass == null) { - throw new UnmarshallingException( - "Can not unpack object because the pojo class is not provided but the object was packed as pojo. " - + "Provide Job result type in JobDescriptor.resultClass." - ); - } - return unpackPojo(unpacker, resultClass); - - case TUPLE_COLLECTION: { - // TODO: IGNITE-24059 Deduplicate with ComputeUtils. - ByteBuffer collectionBuf = unpacker.readBinaryUnsafe().order(ByteOrder.LITTLE_ENDIAN); - int count = collectionBuf.getInt(); - BinaryTupleReader reader = new BinaryTupleReader(count, collectionBuf.slice().order(ByteOrder.LITTLE_ENDIAN)); - - List res = new ArrayList<>(count); - for (int i = 0; i < count; i++) { - ByteBuffer elementBytes = reader.bytesValueAsBuffer(i); - - if (elementBytes == null) { - res.add(null); - continue; - } - - res.add(TupleWithSchemaMarshalling.unmarshal(elementBytes)); - } - - return res; - } - - default: - throw new UnmarshallingException("Unsupported compute job type: " + type); - } - } - - private static Object unpackPojo(ClientMessageUnpacker unpacker, Class pojoClass) { - try { - Object obj = pojoClass.getConstructor().newInstance(); - - fromTuple(obj, TupleWithSchemaMarshalling.unmarshal(unpacker.readBinary())); - - return obj; - } catch (NoSuchMethodException e) { - throw new UnmarshallingException("Class " + pojoClass.getName() + " doesn't have public default constructor. " - + "Add the default constructor or provide Marshaller for " + pojoClass.getName() + " in JobDescriptor.resultMarshaller", - e); - } catch (InvocationTargetException e) { - throw new UnmarshallingException("Constructor has thrown an exception", e); - } catch (InstantiationException e) { - throw new UnmarshallingException("Can't instantiate an object of class " + pojoClass.getName(), e); - } catch (IllegalAccessException e) { - throw new UnmarshallingException("Constructor is inaccessible", e); - } catch (PojoConversionException e) { - throw new UnmarshallingException("Can't unpack object", e); - } + return SharedComputeUtils.unmarshalArgOrResult(holder, marshaller, resultClass); } /** Unpacks compute job argument without marshaller. */ - public static @Nullable Object unpackJobArgumentWithoutMarshaller(ClientMessageUnpacker unpacker) { + public static @Nullable ComputeJobDataHolder unpackJobArgumentWithoutMarshaller(ClientMessageUnpacker unpacker) { if (unpacker.tryUnpackNil()) { return null; } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/compute/ComputeJobDataHolder.java b/modules/client-common/src/main/java/org/apache/ignite/internal/compute/ComputeJobDataHolder.java similarity index 100% rename from modules/core/src/main/java/org/apache/ignite/internal/compute/ComputeJobDataHolder.java rename to modules/client-common/src/main/java/org/apache/ignite/internal/compute/ComputeJobDataHolder.java diff --git a/modules/core/src/main/java/org/apache/ignite/internal/compute/ComputeJobDataType.java b/modules/client-common/src/main/java/org/apache/ignite/internal/compute/ComputeJobDataType.java similarity index 100% rename from modules/core/src/main/java/org/apache/ignite/internal/compute/ComputeJobDataType.java rename to modules/client-common/src/main/java/org/apache/ignite/internal/compute/ComputeJobDataType.java diff --git a/modules/core/src/main/java/org/apache/ignite/internal/compute/JobStateImpl.java b/modules/client-common/src/main/java/org/apache/ignite/internal/compute/JobStateImpl.java similarity index 100% rename from modules/core/src/main/java/org/apache/ignite/internal/compute/JobStateImpl.java rename to modules/client-common/src/main/java/org/apache/ignite/internal/compute/JobStateImpl.java diff --git a/modules/core/src/main/java/org/apache/ignite/internal/compute/JobTaskStatusMapper.java b/modules/client-common/src/main/java/org/apache/ignite/internal/compute/JobTaskStatusMapper.java similarity index 100% rename from modules/core/src/main/java/org/apache/ignite/internal/compute/JobTaskStatusMapper.java rename to modules/client-common/src/main/java/org/apache/ignite/internal/compute/JobTaskStatusMapper.java diff --git a/modules/core/src/main/java/org/apache/ignite/internal/compute/PojoConversionException.java b/modules/client-common/src/main/java/org/apache/ignite/internal/compute/PojoConversionException.java similarity index 100% rename from modules/core/src/main/java/org/apache/ignite/internal/compute/PojoConversionException.java rename to modules/client-common/src/main/java/org/apache/ignite/internal/compute/PojoConversionException.java diff --git a/modules/core/src/main/java/org/apache/ignite/internal/compute/PojoConverter.java b/modules/client-common/src/main/java/org/apache/ignite/internal/compute/PojoConverter.java similarity index 100% rename from modules/core/src/main/java/org/apache/ignite/internal/compute/PojoConverter.java rename to modules/client-common/src/main/java/org/apache/ignite/internal/compute/PojoConverter.java diff --git a/modules/client-common/src/main/java/org/apache/ignite/internal/compute/SharedComputeUtils.java b/modules/client-common/src/main/java/org/apache/ignite/internal/compute/SharedComputeUtils.java new file mode 100644 index 00000000000..1f5d857355b --- /dev/null +++ b/modules/client-common/src/main/java/org/apache/ignite/internal/compute/SharedComputeUtils.java @@ -0,0 +1,261 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.compute; + +import static org.apache.ignite.internal.compute.ComputeJobDataType.MARSHALLED_CUSTOM; +import static org.apache.ignite.internal.compute.ComputeJobDataType.NATIVE; +import static org.apache.ignite.internal.compute.ComputeJobDataType.POJO; +import static org.apache.ignite.internal.compute.ComputeJobDataType.TUPLE; +import static org.apache.ignite.internal.compute.ComputeJobDataType.TUPLE_COLLECTION; +import static org.apache.ignite.internal.compute.PojoConverter.fromTuple; +import static org.apache.ignite.internal.compute.PojoConverter.toTuple; +import static org.apache.ignite.lang.ErrorGroups.Compute.MARSHALLING_TYPE_MISMATCH_ERR; + +import java.lang.reflect.InvocationTargetException; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.List; +import java.util.Set; +import java.util.stream.Collectors; +import org.apache.ignite.compute.ComputeException; +import org.apache.ignite.internal.binarytuple.BinaryTupleBuilder; +import org.apache.ignite.internal.binarytuple.BinaryTupleReader; +import org.apache.ignite.internal.binarytuple.inlineschema.TupleWithSchemaMarshalling; +import org.apache.ignite.internal.client.proto.ClientBinaryTupleUtils; +import org.apache.ignite.internal.util.IgniteUtils; +import org.apache.ignite.marshalling.Marshaller; +import org.apache.ignite.marshalling.MarshallingException; +import org.apache.ignite.marshalling.UnmarshallingException; +import org.apache.ignite.sql.ColumnType; +import org.apache.ignite.table.Tuple; +import org.jetbrains.annotations.Nullable; + +/** + * Compute serialization utils shared between client and embedded APIs. + */ +public class SharedComputeUtils { + private static final Set> NATIVE_TYPES = Arrays.stream(ColumnType.values()) + .map(ColumnType::javaClass) + .collect(Collectors.toUnmodifiableSet()); + + /** + * Marshals the job result using either provided marshaller if not {@code null} or depending on the type of the result either as a + * {@link Tuple}, a native type (see {@link ColumnType}) or a POJO. Wraps the marshalled data with the data type in the + * {@link ComputeJobDataHolder} to be unmarshalled on the client. + * + * @param obj Compute job result. + * @param marshaller Optional result marshaller. + * + * @return Data holder. + */ + @Nullable + public static ComputeJobDataHolder marshalArgOrResult(@Nullable T obj, @Nullable Marshaller marshaller) { + if (obj == null) { + return null; + } + + if (marshaller != null) { + byte[] data = marshaller.marshal(obj); + if (data == null) { + return null; + } + return new ComputeJobDataHolder(MARSHALLED_CUSTOM, data); + } + + if (obj instanceof Tuple) { + Tuple tuple = (Tuple) obj; + return new ComputeJobDataHolder(TUPLE, TupleWithSchemaMarshalling.marshal(tuple)); + } + + if (obj instanceof Collection) { + Collection col = (Collection) obj; + + // Pack entire collection into a single binary blob, starting with the number of elements (4 bytes, little-endian). + BinaryTupleBuilder tupleBuilder = SharedComputeUtils.writeTupleCollection(col); + + ByteBuffer binTupleBytes = tupleBuilder.build(); + + byte[] resArr = new byte[Integer.BYTES + binTupleBytes.remaining()]; + ByteBuffer resBuf = ByteBuffer.wrap(resArr).order(ByteOrder.LITTLE_ENDIAN); + resBuf.putInt(col.size()); + resBuf.put(binTupleBytes); + + return new ComputeJobDataHolder(TUPLE_COLLECTION, resArr); + } + + + if (isNativeType(obj.getClass())) { + // Builder with inline schema. + // Value is represented by 3 tuple elements: type, scale, value. + var builder = new BinaryTupleBuilder(3, 3, false); + ClientBinaryTupleUtils.appendObject(builder, obj); + return new ComputeJobDataHolder(NATIVE, IgniteUtils.byteBufferToByteArray(builder.build())); + } + + try { + // TODO https://issues.apache.org/jira/browse/IGNITE-23320 + Tuple tuple = toTuple(obj); + return new ComputeJobDataHolder(POJO, TupleWithSchemaMarshalling.marshal(tuple)); + } catch (PojoConversionException e) { + throw new MarshallingException("Can't pack object: " + obj, e); + } + } + + /** + * Unmarshals the job argument or result. + * + * @param holder Data holder. + * @param marshaller Optional marshaller. + * @param resultClass Optional result class. + * @param Type of the object. + * @return Unmarshalled object. + */ + public static @Nullable T unmarshalArgOrResult( + @Nullable ComputeJobDataHolder holder, + @Nullable Marshaller marshaller, + @Nullable Class resultClass) { + if (holder == null) { + return null; + } + + ComputeJobDataType type = holder.type(); + if (type != MARSHALLED_CUSTOM && marshaller != null) { + throw new ComputeException( + MARSHALLING_TYPE_MISMATCH_ERR, + "Marshaller is defined on the server, but the argument was not marshalled on the client. " + + "If you want to use default marshalling strategy, " + + "then you should not define your marshaller in the job. " + + "If you would like to use your own marshaller, then double-check " + + "that both of them are defined in the client and in the server." + ); + } + + switch (type) { + case NATIVE: { + var reader = new BinaryTupleReader(3, holder.data()); + return (T) ClientBinaryTupleUtils.readObject(reader, 0); + } + + case TUPLE: + // TODO https://issues.apache.org/jira/browse/IGNITE-23320 + return (T) TupleWithSchemaMarshalling.unmarshal(holder.data()); + + case POJO: + if (resultClass == null) { + throw new ComputeException( + MARSHALLING_TYPE_MISMATCH_ERR, + "JobDescriptor.resultClass is not defined, but the job result is packed as a POJO"); + } + + Tuple tuple = TupleWithSchemaMarshalling.unmarshal(holder.data()); + + return resultClass == Tuple.class + ? (T) tuple + : (T) unmarshalPojo(resultClass, tuple); + + case MARSHALLED_CUSTOM: + if (marshaller == null) { + throw new ComputeException(MARSHALLING_TYPE_MISMATCH_ERR, "Marshaller should be defined on the client"); + } + try { + return (T) marshaller.unmarshal(holder.data()); + } catch (Exception ex) { + throw new ComputeException(MARSHALLING_TYPE_MISMATCH_ERR, "Exception in user-defined marshaller", ex); + } + + case TUPLE_COLLECTION: + return (T) readTupleCollection(ByteBuffer.wrap(holder.data()).order(ByteOrder.LITTLE_ENDIAN)); + + default: + throw new ComputeException(MARSHALLING_TYPE_MISMATCH_ERR, "Unexpected job argument type: " + type); + } + } + + /** + * Unmarshals a POJO from the tuple. + * + * @param pojoType POJO type. + * @param input Tuple. + * @return Unmarshalled POJO. + */ + public static Object unmarshalPojo(Class pojoType, Tuple input) { + try { + Object obj = pojoType.getConstructor().newInstance(); + + fromTuple(obj, input); + + return obj; + } catch (NoSuchMethodException e) { + throw new UnmarshallingException("Class " + pojoType.getName() + " doesn't have public default constructor. " + + "Add the constructor or define argument marshaller in the compute job.", e); + } catch (InvocationTargetException e) { + throw new UnmarshallingException("Constructor has thrown an exception", e); + } catch (InstantiationException e) { + throw new UnmarshallingException("Can't instantiate an object of class " + pojoType.getName(), e); + } catch (IllegalAccessException e) { + throw new UnmarshallingException("Constructor is inaccessible", e); + } catch (PojoConversionException e) { + throw new UnmarshallingException("Can't unpack object", e); + } + } + + private static boolean isNativeType(Class clazz) { + return NATIVE_TYPES.contains(clazz); + } + + private static BinaryTupleBuilder writeTupleCollection(Collection col) { + BinaryTupleBuilder builder = new BinaryTupleBuilder(col.size()); + + for (Object el : col) { + if (el == null) { + builder.appendNull(); + continue; + } + + if (!(el instanceof Tuple)) { + throw new MarshallingException("Can't pack collection: expected Tuple, but got " + el.getClass(), null); + } + + builder.appendBytes(TupleWithSchemaMarshalling.marshal((Tuple) el)); + } + + return builder; + } + + private static List readTupleCollection(ByteBuffer collectionBuf) { + int count = collectionBuf.getInt(); + BinaryTupleReader reader = new BinaryTupleReader(count, collectionBuf.slice().order(ByteOrder.LITTLE_ENDIAN)); + + List res = new ArrayList<>(count); + for (int i = 0; i < count; i++) { + ByteBuffer elementBytes = reader.bytesValueAsBuffer(i); + + if (elementBytes == null) { + res.add(null); + continue; + } + + res.add(TupleWithSchemaMarshalling.unmarshal(elementBytes)); + } + + return res; + } +} diff --git a/modules/core/src/main/java/org/apache/ignite/internal/compute/TaskStateImpl.java b/modules/client-common/src/main/java/org/apache/ignite/internal/compute/TaskStateImpl.java similarity index 100% rename from modules/core/src/main/java/org/apache/ignite/internal/compute/TaskStateImpl.java rename to modules/client-common/src/main/java/org/apache/ignite/internal/compute/TaskStateImpl.java diff --git a/modules/client-common/src/test/java/org/apache/ignite/internal/client/proto/ClientComputeJobPackerUnpackerTest.java b/modules/client-common/src/test/java/org/apache/ignite/internal/client/proto/ClientComputeJobPackerUnpackerTest.java index d93c62dafac..aabf9ff3034 100644 --- a/modules/client-common/src/test/java/org/apache/ignite/internal/client/proto/ClientComputeJobPackerUnpackerTest.java +++ b/modules/client-common/src/test/java/org/apache/ignite/internal/client/proto/ClientComputeJobPackerUnpackerTest.java @@ -44,13 +44,13 @@ import java.util.Set; import java.util.UUID; import java.util.stream.Stream; +import org.apache.ignite.compute.ComputeException; import org.apache.ignite.internal.client.proto.pojo.Pojo; import org.apache.ignite.internal.client.proto.pojo.StaticFieldPojo; import org.apache.ignite.internal.compute.ComputeJobDataHolder; import org.apache.ignite.internal.compute.ComputeJobDataType; import org.apache.ignite.marshalling.Marshaller; import org.apache.ignite.marshalling.MarshallingException; -import org.apache.ignite.marshalling.UnmarshallingException; import org.apache.ignite.table.Tuple; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; @@ -207,9 +207,9 @@ void packWithMarshallerUnpackWithout() { try (var messageUnpacker = messageUnpacker(data)) { // Then the exception is thrown because it is not allowed unpack the marshalled object without marshaller. assertThrows( - UnmarshallingException.class, + ComputeException.class, () -> unpackJobResult(messageUnpacker, null, null), - "Can not unpack object because the marshaller is not provided but the object was packed with marshaller." + "Marshaller should be defined on the client" ); } } @@ -228,9 +228,9 @@ void packByteArrayUnpackStringWithMarshaller() { try (var messageUnpacker = messageUnpacker(data)) { // Then the exception is thrown because it is not allowed to define the marshaller only for the result. assertThrows( - UnmarshallingException.class, + ComputeException.class, () -> unpackJobResult(messageUnpacker, new TestStringMarshaller(), null), - "Can not unpack object because the marshaller is provided but the object was packed without marshaller." + "Marshaller is defined on the server, but the argument was not marshalled on the client" ); } } diff --git a/modules/client-handler/src/main/java/org/apache/ignite/client/handler/requests/compute/ClientComputeExecuteColocatedRequest.java b/modules/client-handler/src/main/java/org/apache/ignite/client/handler/requests/compute/ClientComputeExecuteColocatedRequest.java index f468889bf19..caeba69cfd0 100644 --- a/modules/client-handler/src/main/java/org/apache/ignite/client/handler/requests/compute/ClientComputeExecuteColocatedRequest.java +++ b/modules/client-handler/src/main/java/org/apache/ignite/client/handler/requests/compute/ClientComputeExecuteColocatedRequest.java @@ -30,6 +30,7 @@ import org.apache.ignite.deployment.DeploymentUnit; import org.apache.ignite.internal.client.proto.ClientMessagePacker; import org.apache.ignite.internal.client.proto.ClientMessageUnpacker; +import org.apache.ignite.internal.compute.ComputeJobDataHolder; import org.apache.ignite.internal.compute.IgniteComputeInternal; import org.apache.ignite.internal.network.ClusterService; import org.apache.ignite.table.IgniteTables; @@ -59,11 +60,11 @@ public static CompletableFuture process( List deploymentUnits = in.unpackDeploymentUnits(); String jobClassName = in.unpackString(); JobExecutionOptions options = JobExecutionOptions.builder().priority(in.unpackInt()).maxRetries(in.unpackInt()).build(); - Object args = unpackJobArgumentWithoutMarshaller(in); + ComputeJobDataHolder args = unpackJobArgumentWithoutMarshaller(in); out.packInt(table.schemaView().lastKnownSchemaVersion()); - CompletableFuture> jobExecutionFut = compute.submitColocatedInternal( + CompletableFuture> jobExecutionFut = compute.submitColocatedInternal( table, keyTuple, deploymentUnits, @@ -72,7 +73,7 @@ public static CompletableFuture process( null, args); - var jobExecution = compute.wrapJobExecutionFuture(jobExecutionFut); + JobExecution jobExecution = compute.wrapJobExecutionFuture(jobExecutionFut); sendResultAndState(jobExecution, notificationSender); diff --git a/modules/client-handler/src/main/java/org/apache/ignite/client/handler/requests/compute/ClientComputeExecuteRequest.java b/modules/client-handler/src/main/java/org/apache/ignite/client/handler/requests/compute/ClientComputeExecuteRequest.java index bf6a621a959..8342398fbaf 100644 --- a/modules/client-handler/src/main/java/org/apache/ignite/client/handler/requests/compute/ClientComputeExecuteRequest.java +++ b/modules/client-handler/src/main/java/org/apache/ignite/client/handler/requests/compute/ClientComputeExecuteRequest.java @@ -32,6 +32,7 @@ import org.apache.ignite.internal.client.proto.ClientComputeJobPacker; import org.apache.ignite.internal.client.proto.ClientMessagePacker; import org.apache.ignite.internal.client.proto.ClientMessageUnpacker; +import org.apache.ignite.internal.compute.ComputeJobDataHolder; import org.apache.ignite.internal.compute.IgniteComputeInternal; import org.apache.ignite.internal.compute.MarshallerProvider; import org.apache.ignite.internal.network.ClusterService; @@ -65,9 +66,9 @@ public static CompletableFuture process( List deploymentUnits = in.unpackDeploymentUnits(); String jobClassName = in.unpackString(); JobExecutionOptions options = JobExecutionOptions.builder().priority(in.unpackInt()).maxRetries(in.unpackInt()).build(); - Object arg = unpackJobArgumentWithoutMarshaller(in); + ComputeJobDataHolder arg = unpackJobArgumentWithoutMarshaller(in); - JobExecution execution = compute.executeAsyncWithFailover( + JobExecution execution = compute.executeAsyncWithFailover( candidates, deploymentUnits, jobClassName, options, null, arg ); sendResultAndState(execution, notificationSender); @@ -102,8 +103,8 @@ private static Set unpackCandidateNodes(ClientMessageUnpacker in, C return nodes; } - static CompletableFuture sendResultAndState( - JobExecution execution, + static CompletableFuture sendResultAndState( + JobExecution execution, NotificationSender notificationSender ) { return execution.resultAsync().whenComplete((val, err) -> @@ -115,7 +116,7 @@ static CompletableFuture sendResultAndState( }, err))); } - private static @Nullable Marshaller extractMarshaller(JobExecution e) { + private static @Nullable Marshaller extractMarshaller(JobExecution e) { if (e instanceof MarshallerProvider) { return ((MarshallerProvider) e).resultMarshaller(); } diff --git a/modules/client/src/test/java/org/apache/ignite/client/fakes/FakeCompute.java b/modules/client/src/test/java/org/apache/ignite/client/fakes/FakeCompute.java index 91e5c055780..39b9cbeeedd 100644 --- a/modules/client/src/test/java/org/apache/ignite/client/fakes/FakeCompute.java +++ b/modules/client/src/test/java/org/apache/ignite/client/fakes/FakeCompute.java @@ -54,11 +54,13 @@ import org.apache.ignite.compute.TaskState; import org.apache.ignite.compute.task.TaskExecution; import org.apache.ignite.deployment.DeploymentUnit; +import org.apache.ignite.internal.compute.ComputeJobDataHolder; import org.apache.ignite.internal.compute.ComputeUtils; import org.apache.ignite.internal.compute.IgniteComputeInternal; import org.apache.ignite.internal.compute.JobExecutionContextImpl; import org.apache.ignite.internal.compute.JobStateImpl; import org.apache.ignite.internal.compute.MarshallerProvider; +import org.apache.ignite.internal.compute.SharedComputeUtils; import org.apache.ignite.internal.compute.TaskStateImpl; import org.apache.ignite.internal.compute.loader.JobClassLoader; import org.apache.ignite.internal.table.TableViewInternal; @@ -94,16 +96,16 @@ public FakeCompute(String nodeName, Ignite ignite) { } @Override - public JobExecution executeAsyncWithFailover( + public JobExecution executeAsyncWithFailover( Set nodes, List units, String jobClassName, JobExecutionOptions options, @Nullable CancellationToken cancellationToken, - Object args) { + @Nullable ComputeJobDataHolder args) { if (Objects.equals(jobClassName, GET_UNITS)) { String unitString = units.stream().map(DeploymentUnit::render).collect(Collectors.joining(",")); - return completedExecution((R) unitString); + return completedExecution(unitString); } try { @@ -119,39 +121,75 @@ public JobExecution executeAsyncWithFailover( if (jobClassName.startsWith("org.apache.ignite")) { JobClassLoader jobClassLoader = new JobClassLoader(List.of(), new URL[]{}, this.getClass().getClassLoader()); - Class> jobClass = ComputeUtils.jobClass(jobClassLoader, jobClassName); - ComputeJob job = ComputeUtils.instantiateJob(jobClass); - CompletableFuture jobFut = job.executeAsync( - new JobExecutionContextImpl(ignite, new AtomicBoolean(), this.getClass().getClassLoader()), args); + Class> jobClass = ComputeUtils.jobClass(jobClassLoader, jobClassName); + ComputeJob job = ComputeUtils.instantiateJob(jobClass); + CompletableFuture jobFut = job.executeAsync( + new JobExecutionContextImpl(ignite, new AtomicBoolean(), this.getClass().getClassLoader()), + SharedComputeUtils.unmarshalArgOrResult(args, null, null)); return jobExecution(jobFut != null ? jobFut : nullCompletedFuture()); } var future0 = future; - return jobExecution(future0 != null ? future0 : completedFuture((R) nodeName)); + return jobExecution(future0 != null ? future0 : completedFuture(SharedComputeUtils.marshalArgOrResult(nodeName, null))); } /** {@inheritDoc} */ @Override - public CompletableFuture> submitColocatedInternal( + public CompletableFuture> submitColocatedInternal( TableViewInternal table, Tuple key, List units, String jobClassName, JobExecutionOptions options, @Nullable CancellationToken cancellationToken, - Object args + ComputeJobDataHolder args ) { - return completedFuture(jobExecution(future != null ? future : completedFuture((R) nodeName))); + return completedFuture(jobExecution(future != null + ? future + : completedFuture(SharedComputeUtils.marshalArgOrResult(nodeName, null)))); } - private JobExecution submit(JobTarget target, JobDescriptor descriptor, @Nullable CancellationToken cancellationToken, + private JobExecution submit( + JobTarget target, + JobDescriptor descriptor, + @Nullable CancellationToken cancellationToken, @Nullable T args) { if (target instanceof AnyNodeJobTarget) { Set nodes = ((AnyNodeJobTarget) target).nodes(); - return executeAsyncWithFailover( - nodes, descriptor.units(), descriptor.jobClassName(), descriptor.options(), cancellationToken, args + + JobExecution internalExecution = executeAsyncWithFailover( + nodes, + descriptor.units(), + descriptor.jobClassName(), + descriptor.options(), + cancellationToken, + SharedComputeUtils.marshalArgOrResult(args, null) ); + + return new JobExecution<>() { + @Override + public CompletableFuture resultAsync() { + return internalExecution.resultAsync() + .thenApply(r -> SharedComputeUtils.unmarshalArgOrResult( + r, descriptor.resultMarshaller(), descriptor.resultClass())); + } + + @Override + public CompletableFuture<@Nullable JobState> stateAsync() { + return internalExecution.stateAsync(); + } + + @Override + public CompletableFuture<@Nullable Boolean> cancelAsync() { + return internalExecution.cancelAsync(); + } + + @Override + public CompletableFuture<@Nullable Boolean> changePriorityAsync(int newPriority) { + return internalExecution.cancelAsync(); + } + }; } else if (target instanceof ColocatedJobTarget) { return jobExecution(future != null ? future : completedFuture((R) nodeName)); } else { @@ -201,11 +239,11 @@ public R executeMapReduce(TaskDescriptor taskDescriptor, @Nullable return sync(executeMapReduceAsync(taskDescriptor, cancellationToken, arg)); } - private JobExecution completedExecution(R result) { - return jobExecution(completedFuture(result)); + private JobExecution completedExecution(R result) { + return jobExecution(completedFuture(SharedComputeUtils.marshalArgOrResult(result, null))); } - private JobExecution jobExecution(CompletableFuture result) { + private JobExecution jobExecution(CompletableFuture result) { UUID jobId = UUID.randomUUID(); JobState state = JobStateImpl.builder() @@ -221,7 +259,10 @@ private JobExecution jobExecution(CompletableFuture result) { JobState newState = JobStateImpl.toBuilder(state).status(status).finishTime(Instant.now()).build(); jobStates.put(jobId, newState); }); - return new FakeJobExecution<>(result, jobId); + + return new FakeJobExecution<>(result.thenApply(r -> r instanceof ComputeJobDataHolder + ? (ComputeJobDataHolder) r + : SharedComputeUtils.marshalArgOrResult(r, null)), jobId); } private class FakeJobExecution implements JobExecution, MarshallerProvider { diff --git a/modules/compute/build.gradle b/modules/compute/build.gradle index a32ad834fd1..6a9b873a168 100644 --- a/modules/compute/build.gradle +++ b/modules/compute/build.gradle @@ -63,6 +63,7 @@ dependencies { integrationTestImplementation project(':ignite-catalog') integrationTestImplementation project(':ignite-client') integrationTestImplementation project(':ignite-system-view-api') + integrationTestImplementation project(':ignite-client-common') integrationTestImplementation testFixtures(project(':ignite-core')) integrationTestImplementation testFixtures(project(':ignite-runner')) } diff --git a/modules/compute/src/integrationTest/java/org/apache/ignite/internal/compute/ItComputeBaseTest.java b/modules/compute/src/integrationTest/java/org/apache/ignite/internal/compute/ItComputeBaseTest.java index 2a88dbe753f..3736855d094 100644 --- a/modules/compute/src/integrationTest/java/org/apache/ignite/internal/compute/ItComputeBaseTest.java +++ b/modules/compute/src/integrationTest/java/org/apache/ignite/internal/compute/ItComputeBaseTest.java @@ -157,10 +157,10 @@ void executesJobLocally() { String result = compute().execute( JobTarget.node(clusterNode(entryNode)), - JobDescriptor.builder(concatJobClass()).units(units()).build(), - new Object[]{"a", 42}); + JobDescriptor.builder(toStringJobClass()).units(units()).build(), + 42); - assertThat(result, is("a42")); + assertThat(result, is("42")); } @Test @@ -169,10 +169,10 @@ void executesJobLocallyAsync() { JobExecution execution = compute().submit( JobTarget.node(clusterNode(entryNode)), - JobDescriptor.builder(concatJobClass()).units(units()).build(), - new Object[] {"a", 42}); + JobDescriptor.builder(toStringJobClass()).units(units()).build(), + 42); - assertThat(execution.resultAsync(), willBe("a42")); + assertThat(execution.resultAsync(), willBe("42")); assertThat(execution.stateAsync(), willBe(jobStateWithStatus(COMPLETED))); assertThat(execution.cancelAsync(), willBe(false)); } @@ -181,20 +181,20 @@ void executesJobLocallyAsync() { void executesJobOnRemoteNodes() { String result = compute().execute( JobTarget.anyNode(clusterNode(node(1)), clusterNode(node(2))), - JobDescriptor.builder(concatJobClass()).units(units()).build(), - new Object[]{"a", 42}); + JobDescriptor.builder(toStringJobClass()).units(units()).build(), + 42); - assertThat(result, is("a42")); + assertThat(result, is("42")); } @Test void executesJobOnRemoteNodesAsync() { JobExecution execution = compute().submit( JobTarget.anyNode(clusterNode(node(1)), clusterNode(node(2))), - JobDescriptor.builder(concatJobClass()).units(units()).build(), - new Object[]{"a", 42}); + JobDescriptor.builder(toStringJobClass()).units(units()).build(), + 42); - assertThat(execution.resultAsync(), willBe("a42")); + assertThat(execution.resultAsync(), willBe("42")); assertThat(execution.stateAsync(), willBe(jobStateWithStatus(COMPLETED))); assertThat(execution.cancelAsync(), willBe(false)); } @@ -290,14 +290,14 @@ void broadcastsJobWithArgumentsAsync() { Map> results = compute().submitBroadcast( Set.of(clusterNode(entryNode), clusterNode(node(1)), clusterNode(node(2))), - JobDescriptor.builder(concatJobClass()).units(units()).build(), - new Object[] {"a", 42}); + JobDescriptor.builder(toStringJobClass()).units(units()).build(), + 42); assertThat(results, is(aMapWithSize(3))); for (int i = 0; i < 3; i++) { ClusterNode node = clusterNode(node(i)); JobExecution execution = results.get(node); - assertThat(execution.resultAsync(), willBe("a42")); + assertThat(execution.resultAsync(), willBe("42")); assertThat(execution.stateAsync(), willBe(jobStateWithStatus(COMPLETED))); assertThat(execution.cancelAsync(), willBe(false)); } @@ -691,12 +691,12 @@ private static Stream tupleCollections() { ).map(Arguments::of); } - static String concatJobClassName() { - return ConcatJob.class.getName(); + static String toStringJobClassName() { + return ToStringJob.class.getName(); } - private static Class concatJobClass() { - return ConcatJob.class; + private static Class toStringJobClass() { + return ToStringJob.class; } private static String getNodeNameJobClassName() { diff --git a/modules/compute/src/integrationTest/java/org/apache/ignite/internal/compute/ItComputeTestClient.java b/modules/compute/src/integrationTest/java/org/apache/ignite/internal/compute/ItComputeTestClient.java index a3ab1a64f31..83b8a25bb4b 100644 --- a/modules/compute/src/integrationTest/java/org/apache/ignite/internal/compute/ItComputeTestClient.java +++ b/modules/compute/src/integrationTest/java/org/apache/ignite/internal/compute/ItComputeTestClient.java @@ -23,7 +23,6 @@ import org.apache.ignite.compute.IgniteCompute; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.Disabled; /** * Integration tests for Compute functionality using thin client API. @@ -55,34 +54,4 @@ protected IgniteCompute compute() { void cancelsNotCancellableJob(boolean local) { // No-op. Embedded-specific. } - - @Override - @Disabled("IGNITE-24059 Compute argument serialization is inconsistent in embedded and thin client") - void executesJobLocally() { - super.executesJobLocally(); - } - - @Override - @Disabled("IGNITE-24059 Compute argument serialization is inconsistent in embedded and thin client") - void executesJobLocallyAsync() { - super.executesJobLocallyAsync(); - } - - @Override - @Disabled("IGNITE-24059 Compute argument serialization is inconsistent in embedded and thin client") - void broadcastsJobWithArgumentsAsync() { - super.broadcastsJobWithArgumentsAsync(); - } - - @Override - @Disabled("IGNITE-24059 Compute argument serialization is inconsistent in embedded and thin client") - void executesJobOnRemoteNodes() { - super.executesJobOnRemoteNodes(); - } - - @Override - @Disabled("IGNITE-24059 Compute argument serialization is inconsistent in embedded and thin client") - void executesJobOnRemoteNodesAsync() { - super.executesJobOnRemoteNodesAsync(); - } } diff --git a/modules/compute/src/integrationTest/java/org/apache/ignite/internal/compute/ItComputeTestEmbedded.java b/modules/compute/src/integrationTest/java/org/apache/ignite/internal/compute/ItComputeTestEmbedded.java index 99b01ad1307..b10c1b945c4 100644 --- a/modules/compute/src/integrationTest/java/org/apache/ignite/internal/compute/ItComputeTestEmbedded.java +++ b/modules/compute/src/integrationTest/java/org/apache/ignite/internal/compute/ItComputeTestEmbedded.java @@ -63,6 +63,7 @@ import org.apache.ignite.network.ClusterNode; import org.apache.ignite.table.KeyValueView; import org.apache.ignite.table.Table; +import org.jetbrains.annotations.Nullable; import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.Arguments; @@ -71,32 +72,34 @@ /** * Integration tests for Compute functionality in embedded Ignite mode. */ -@SuppressWarnings({"NewClassNamingConvention"}) +@SuppressWarnings("NewClassNamingConvention") class ItComputeTestEmbedded extends ItComputeBaseTest { - @Override protected List units() { return List.of(); } + @SuppressWarnings("AssignmentToStaticFieldFromInstanceMethod") @Test void changeJobPriorityLocally() { Ignite entryNode = node(0); JobTarget jobTarget = JobTarget.node(clusterNode(entryNode)); CountDownLatch countDownLatch = new CountDownLatch(1); - JobDescriptor job = JobDescriptor.builder(WaitLatchJob.class).units(units()).build(); + WaitLatchJob.latches = new CountDownLatch[]{countDownLatch, new CountDownLatch(1)}; + + JobDescriptor job = JobDescriptor.builder(WaitLatchJob.class).units(units()).build(); // Start 1 task in executor with 1 thread - JobExecution execution1 = entryNode.compute().submit(jobTarget, job, countDownLatch); + JobExecution execution1 = entryNode.compute().submit(jobTarget, job, 0); await().until(execution1::stateAsync, willBe(jobStateWithStatus(EXECUTING))); // Start one more task - JobExecution execution2 = entryNode.compute().submit(jobTarget, job, new CountDownLatch(1)); + JobExecution execution2 = entryNode.compute().submit(jobTarget, job, 1); await().until(execution2::stateAsync, willBe(jobStateWithStatus(QUEUED))); // Start third task - JobExecution execution3 = entryNode.compute().submit(jobTarget, job, countDownLatch); + JobExecution execution3 = entryNode.compute().submit(jobTarget, job, 0); await().until(execution3::stateAsync, willBe(jobStateWithStatus(QUEUED))); // Task 2 and 3 are not completed, in queued state @@ -120,32 +123,37 @@ void changeJobPriorityLocally() { assertThat(execution2.cancelAsync(), willBe(true)); } + @SuppressWarnings("AssignmentToStaticFieldFromInstanceMethod") @Test void executesJobLocallyWithOptions() { Ignite entryNode = node(0); JobTarget jobTarget = JobTarget.node(clusterNode(entryNode)); CountDownLatch countDownLatch = new CountDownLatch(1); - JobDescriptor job = JobDescriptor.builder(WaitLatchJob.class).units(units()).build(); + WaitLatchJob.latches = new CountDownLatch[]{countDownLatch, new CountDownLatch(1)}; + + JobDescriptor job = JobDescriptor.builder(WaitLatchJob.class).units(units()).build(); // Start 1 task in executor with 1 thread - JobExecution execution1 = entryNode.compute().submit(jobTarget, job, countDownLatch); + JobExecution execution1 = entryNode.compute().submit(jobTarget, job, 0); await().until(execution1::stateAsync, willBe(jobStateWithStatus(EXECUTING))); // Start one more task - JobExecution execution2 = entryNode.compute().submit(jobTarget, job, new CountDownLatch(1)); + JobExecution execution2 = entryNode.compute().submit(jobTarget, job, 1); await().until(execution2::stateAsync, willBe(jobStateWithStatus(QUEUED))); // Start third task it should be before task2 in the queue due to higher priority in options JobExecutionOptions options = JobExecutionOptions.builder().priority(1).maxRetries(2).build(); - JobExecution execution3 = entryNode.compute().submit( + WaitLatchThrowExceptionOnFirstExecutionJob.latch = countDownLatch; + + JobExecution execution3 = entryNode.compute().submit( jobTarget, JobDescriptor.builder(WaitLatchThrowExceptionOnFirstExecutionJob.class) .units(units()) .options(options) .build(), - countDownLatch); + null); await().until(execution3::stateAsync, willBe(jobStateWithStatus(QUEUED))); // Task 1 and 2 are not competed, in queue state @@ -172,30 +180,34 @@ void executesJobLocallyWithOptions() { assertThat(execution2.cancelAsync(), willBe(true)); } + @SuppressWarnings("AssignmentToStaticFieldFromInstanceMethod") @Test void shouldNotConvertIgniteException() { Ignite entryNode = node(0); IgniteException exception = new IgniteException(INTERNAL_ERR, "Test exception"); + CustomFailingJob.th = exception; IgniteException ex = assertThrows(IgniteException.class, () -> entryNode.compute().execute( JobTarget.node(clusterNode(entryNode)), JobDescriptor.builder(CustomFailingJob.class).units(units()).build(), - exception)); + null)); assertPublicException(ex, exception.code(), exception.getMessage()); } + @SuppressWarnings("AssignmentToStaticFieldFromInstanceMethod") @Test void shouldNotConvertIgniteCheckedException() { Ignite entryNode = node(0); IgniteCheckedException exception = new IgniteCheckedException(INTERNAL_ERR, "Test exception"); + CustomFailingJob.th = exception; IgniteCheckedException ex = assertThrows(IgniteCheckedException.class, () -> entryNode.compute().execute( JobTarget.node(clusterNode(entryNode)), JobDescriptor.builder(CustomFailingJob.class).units(units()).build(), - exception)); + null)); assertPublicCheckedException(ex, exception.code(), exception.getMessage()); } @@ -209,15 +221,18 @@ private static Stream privateExceptions() { ); } + @SuppressWarnings("AssignmentToStaticFieldFromInstanceMethod") @ParameterizedTest @MethodSource("privateExceptions") void shouldConvertToComputeException(Throwable throwable) { Ignite entryNode = node(0); + CustomFailingJob.th = throwable; + IgniteException ex = assertThrows(IgniteException.class, () -> entryNode.compute().execute( JobTarget.node(clusterNode(entryNode)), JobDescriptor.builder(CustomFailingJob.class).units(units()).build(), - throwable)); + null)); assertComputeException(ex, throwable); } @@ -284,18 +299,22 @@ private Stream targetNodeIndexes() { return IntStream.range(0, initialNodes()).mapToObj(Arguments::of); } - private static class CustomFailingJob implements ComputeJob { + private static class CustomFailingJob implements ComputeJob { + static Throwable th; + @Override - public CompletableFuture executeAsync(JobExecutionContext context, Throwable th) { + public CompletableFuture executeAsync(JobExecutionContext context, Void arg) { throw ExceptionUtils.sneakyThrow(th); } } - private static class WaitLatchJob implements ComputeJob { + private static class WaitLatchJob implements ComputeJob { + private static CountDownLatch[] latches; + @Override - public CompletableFuture executeAsync(JobExecutionContext context, CountDownLatch latch) { + public @Nullable CompletableFuture executeAsync(JobExecutionContext context, Integer latchId) { try { - latch.await(); + latches[latchId].await(); } catch (InterruptedException e) { Thread.currentThread().interrupt(); } @@ -303,11 +322,12 @@ public CompletableFuture executeAsync(JobExecutionContext context, Count } } - private static class WaitLatchThrowExceptionOnFirstExecutionJob implements ComputeJob { + private static class WaitLatchThrowExceptionOnFirstExecutionJob implements ComputeJob { + private static CountDownLatch latch; static final AtomicInteger counter = new AtomicInteger(0); @Override - public CompletableFuture executeAsync(JobExecutionContext context, CountDownLatch latch) { + public @Nullable CompletableFuture executeAsync(JobExecutionContext context, Void arg) { try { latch.await(); if (counter.incrementAndGet() == 1) { diff --git a/modules/compute/src/integrationTest/java/org/apache/ignite/internal/compute/ItComputeTestStandalone.java b/modules/compute/src/integrationTest/java/org/apache/ignite/internal/compute/ItComputeTestStandalone.java index 5620c49db5e..8159eb7b742 100644 --- a/modules/compute/src/integrationTest/java/org/apache/ignite/internal/compute/ItComputeTestStandalone.java +++ b/modules/compute/src/integrationTest/java/org/apache/ignite/internal/compute/ItComputeTestStandalone.java @@ -88,15 +88,15 @@ void executesJobWithNonExistingUnit() { List nonExistingUnits = List.of(new DeploymentUnit("non-existing", "1.0.0")); CompletableFuture result = entryNode.compute().executeAsync( JobTarget.node(clusterNode(entryNode)), - JobDescriptor.builder(concatJobClassName()).units(nonExistingUnits).build(), - new Object[]{"a", 42}); + JobDescriptor.builder(toStringJobClassName()).units(nonExistingUnits).build(), + null); CompletionException ex0 = assertThrows(CompletionException.class, result::join); assertComputeException( ex0, ClassNotFoundException.class, - "org.apache.ignite.internal.compute.ConcatJob. Deployment unit non-existing:1.0.0 doesn't exist" + "org.apache.ignite.internal.compute.ToStringJob. Deployment unit non-existing:1.0.0 doesn't exist" ); } diff --git a/modules/compute/src/jobs/java/org/apache/ignite/internal/compute/ConcatJob.java b/modules/compute/src/jobs/java/org/apache/ignite/internal/compute/ToStringJob.java similarity index 75% rename from modules/compute/src/jobs/java/org/apache/ignite/internal/compute/ConcatJob.java rename to modules/compute/src/jobs/java/org/apache/ignite/internal/compute/ToStringJob.java index 9cf96f47452..de38ccace8c 100644 --- a/modules/compute/src/jobs/java/org/apache/ignite/internal/compute/ConcatJob.java +++ b/modules/compute/src/jobs/java/org/apache/ignite/internal/compute/ToStringJob.java @@ -19,19 +19,15 @@ import static java.util.concurrent.CompletableFuture.completedFuture; -import java.util.Arrays; import java.util.concurrent.CompletableFuture; -import java.util.stream.Collectors; import org.apache.ignite.compute.ComputeJob; import org.apache.ignite.compute.JobExecutionContext; -/** Compute job that concatenates the string representation of its arguments. */ -public class ConcatJob implements ComputeJob { +/** Compute job that returns the string representation of its argument. */ +public class ToStringJob implements ComputeJob { @Override - public CompletableFuture executeAsync(JobExecutionContext context, Object... input) { - return completedFuture(Arrays.stream(input) - .map(Object::toString) - .collect(Collectors.joining())); + public CompletableFuture executeAsync(JobExecutionContext context, Object input) { + return completedFuture(input == null ? null : input.toString()); } } diff --git a/modules/compute/src/main/java/org/apache/ignite/internal/compute/ComputeComponent.java b/modules/compute/src/main/java/org/apache/ignite/internal/compute/ComputeComponent.java index ad320d95291..60ba436efed 100644 --- a/modules/compute/src/main/java/org/apache/ignite/internal/compute/ComputeComponent.java +++ b/modules/compute/src/main/java/org/apache/ignite/internal/compute/ComputeComponent.java @@ -44,37 +44,16 @@ public interface ComputeComponent extends IgniteComponent { * @param jobClassName Name of the job class. * @param cancellationToken Cancellation token or {@code null}. * @param arg Job args. - * @param Job result type. * @return Job execution object. */ - JobExecution executeLocally( + JobExecution executeLocally( ExecutionOptions options, List units, String jobClassName, @Nullable CancellationToken cancellationToken, - @Nullable T arg + @Nullable ComputeJobDataHolder arg ); - /** - * Executes a job of the given class on the current node. - * - * - * @param options Job execution options. - * @param units Deployment units which will be loaded for execution. - * @param jobClassName Name of the job class. - * @param arg Job args. - * @param Job result type. - * @return Job execution object. - */ - default JobExecution executeLocally( - ExecutionOptions options, - List units, - String jobClassName, - @Nullable T arg - ) { - return executeLocally(options, units, jobClassName, null, arg); - } - /** * Executes a job of the given class on a remote node. * @@ -84,16 +63,15 @@ default JobExecution executeLocally( * @param jobClassName Name of the job class. * @param cancellationToken Cancellation token or {@code null}. * @param arg Job args. - * @param Job result type. * @return Job execution object. */ - JobExecution executeRemotely( + JobExecution executeRemotely( ExecutionOptions options, ClusterNode remoteNode, List units, String jobClassName, @Nullable CancellationToken cancellationToken, - T arg + @Nullable ComputeJobDataHolder arg ); /** @@ -107,17 +85,16 @@ JobExecution executeRemotely( * @param jobClassName Name of the job class. * @param cancellationToken Cancellation token or {@code null}. * @param arg Job args. - * @param Job result type. * @return Job execution object. */ - JobExecution executeRemotelyWithFailover( + JobExecution executeRemotelyWithFailover( ClusterNode remoteNode, NextWorkerSelector nextWorkerSelector, List units, String jobClassName, ExecutionOptions options, @Nullable CancellationToken cancellationToken, - @Nullable T arg + @Nullable ComputeJobDataHolder arg ); /** diff --git a/modules/compute/src/main/java/org/apache/ignite/internal/compute/ComputeComponentImpl.java b/modules/compute/src/main/java/org/apache/ignite/internal/compute/ComputeComponentImpl.java index c68056c66e2..f2c613d4a1f 100644 --- a/modules/compute/src/main/java/org/apache/ignite/internal/compute/ComputeComponentImpl.java +++ b/modules/compute/src/main/java/org/apache/ignite/internal/compute/ComputeComponentImpl.java @@ -122,15 +122,15 @@ public ComputeComponentImpl( /** {@inheritDoc} */ @Override - public JobExecution executeLocally( + public JobExecution executeLocally( ExecutionOptions options, List units, String jobClassName, @Nullable CancellationToken cancellationToken, - I arg + @Nullable ComputeJobDataHolder arg ) { if (!busyLock.enterBusy()) { - return new DelegatingJobExecution<>( + return new DelegatingJobExecution( failedFuture(new IgniteInternalException(NODE_STOPPING_ERR, new NodeStoppingException())) ); } @@ -138,10 +138,10 @@ public JobExecution executeLocally( try { CompletableFuture classLoaderFut = jobContextManager.acquireClassLoader(units); - CompletableFuture> future = + CompletableFuture> future = mapClassLoaderExceptions(classLoaderFut, jobClassName) .thenApply(context -> { - JobExecutionInternal execution = execJob(context, options, jobClassName, arg); + JobExecutionInternal execution = execJob(context, options, jobClassName, arg); execution.resultAsync().whenComplete((result, e) -> context.close()); inFlightFutures.registerFuture(execution.resultAsync()); return execution; @@ -150,7 +150,7 @@ public JobExecution executeLocally( inFlightFutures.registerFuture(future); inFlightFutures.registerFuture(classLoaderFut); - JobExecution result = new DelegatingJobExecution<>(future); + JobExecution result = new DelegatingJobExecution(future); if (cancellationToken != null) { CancelHandleHelper.addCancelAction(cancellationToken, classLoaderFut); @@ -191,6 +191,7 @@ public TaskExecution executeTask( inFlightFutures.registerFuture(taskFuture); DelegatingTaskExecution result = new DelegatingTaskExecution<>(taskFuture); + result.idAsync().thenAccept(jobId -> executionManager.addExecution(jobId, new TaskToJobExecutionWrapper<>(result))); return result; } finally { @@ -200,28 +201,30 @@ public TaskExecution executeTask( /** {@inheritDoc} */ @Override - public JobExecution executeRemotely( + public JobExecution executeRemotely( ExecutionOptions options, ClusterNode remoteNode, List units, String jobClassName, @Nullable CancellationToken cancellationToken, - T arg + @Nullable ComputeJobDataHolder arg ) { if (!busyLock.enterBusy()) { - return new DelegatingJobExecution<>( + return new DelegatingJobExecution( failedFuture(new IgniteInternalException(NODE_STOPPING_ERR, new NodeStoppingException())) ); } try { CompletableFuture jobIdFuture = messaging.remoteExecuteRequestAsync(options, remoteNode, units, jobClassName, arg); - CompletableFuture resultFuture = jobIdFuture.thenCompose(jobId -> messaging.remoteJobResultRequestAsync(remoteNode, jobId)); + CompletableFuture resultFuture = jobIdFuture.thenCompose( + jobId -> messaging.remoteJobResultRequestAsync(remoteNode, jobId)); inFlightFutures.registerFuture(jobIdFuture); inFlightFutures.registerFuture(resultFuture); - JobExecution result = new RemoteJobExecution<>(remoteNode, jobIdFuture, resultFuture, inFlightFutures, messaging); + JobExecution result = new RemoteJobExecution<>( + remoteNode, jobIdFuture, resultFuture, inFlightFutures, messaging); if (cancellationToken != null) { CancelHandleHelper.addCancelAction(cancellationToken, result::cancelAsync, result.resultAsync()); @@ -235,16 +238,16 @@ public JobExecution executeRemotely( } @Override - public JobExecution executeRemotelyWithFailover( + public JobExecution executeRemotelyWithFailover( ClusterNode remoteNode, NextWorkerSelector nextWorkerSelector, List units, String jobClassName, ExecutionOptions options, @Nullable CancellationToken cancellationToken, - @Nullable T arg + @Nullable ComputeJobDataHolder arg ) { - JobExecution result = (JobExecution) new ComputeJobFailover<>( + JobExecution result = new ComputeJobFailover( this, logicalTopologyService, topologyService, remoteNode, nextWorkerSelector, failoverExecutor, units, jobClassName, options, cancellationToken, arg @@ -293,7 +296,8 @@ public CompletableFuture> statesAsync() { @Override public CompletableFuture startAsync(ComponentContext componentContext) { executor.start(); - messaging.start(this::executeLocally); + messaging.start((options, units, jobClassName, arg) -> + executeLocally(options, units, jobClassName, null, arg)); executionManager.start(); computeViewProvider.init(executionManager); @@ -320,7 +324,11 @@ public CompletableFuture stopAsync(ComponentContext componentContext) { } - private JobExecutionInternal execJob(JobContext context, ExecutionOptions options, String jobClassName, T arg) { + private JobExecutionInternal execJob( + JobContext context, + ExecutionOptions options, + String jobClassName, + @Nullable ComputeJobDataHolder arg) { try { return executor.executeJob(options, jobClass(context.classLoader(), jobClassName), context.classLoader(), arg); } catch (Throwable e) { diff --git a/modules/compute/src/main/java/org/apache/ignite/internal/compute/ComputeJobFailover.java b/modules/compute/src/main/java/org/apache/ignite/internal/compute/ComputeJobFailover.java index ae531a1ee02..87f36789736 100644 --- a/modules/compute/src/main/java/org/apache/ignite/internal/compute/ComputeJobFailover.java +++ b/modules/compute/src/main/java/org/apache/ignite/internal/compute/ComputeJobFailover.java @@ -43,10 +43,8 @@ *

If you want to execute a job on node1 and use node2 and node3 as failover candidates, * then you should create an instance of this class with workerNode = node1, failoverCandidates = [node2, node3] as arguments and call * {@link #failSafeExecute()}. - * - * @param the type of the result of the job. */ -class ComputeJobFailover { +class ComputeJobFailover { private static final IgniteLogger LOG = Loggers.forClass(ComputeJobFailover.class); /** @@ -84,7 +82,7 @@ class ComputeJobFailover { /** * Context of the called job. Captures deployment units, jobClassName and arguments. */ - private final RemoteExecutionContext jobContext; + private final RemoteExecutionContext jobContext; /** Cancellation token. */ @Nullable private final CancellationToken cancellationToken; @@ -102,7 +100,7 @@ class ComputeJobFailover { * @param jobClassName the name of the job class. * @param executionOptions execution options like priority or max retries. * @param cancellationToken Cancellation token or {@code null}. - * @param args the arguments of the job. + * @param arg the arguments of the job. */ ComputeJobFailover( ComputeComponent computeComponent, @@ -115,14 +113,14 @@ class ComputeJobFailover { String jobClassName, ExecutionOptions executionOptions, @Nullable CancellationToken cancellationToken, - Object args + @Nullable ComputeJobDataHolder arg ) { this.computeComponent = computeComponent; this.runningWorkerNode = new AtomicReference<>(workerNode); this.logicalTopologyService = logicalTopologyService; this.topologyService = topologyService; this.nextWorkerSelector = nextWorkerSelector; - this.jobContext = new RemoteExecutionContext<>(units, jobClassName, executionOptions, args); + this.jobContext = new RemoteExecutionContext(units, jobClassName, executionOptions, arg); this.executor = executor; this.cancellationToken = cancellationToken; } @@ -132,8 +130,8 @@ class ComputeJobFailover { * * @return JobExecution with the result of the job and the status of the job. */ - JobExecution failSafeExecute() { - JobExecution jobExecution = launchJobOn(runningWorkerNode.get()); + JobExecution failSafeExecute() { + JobExecution jobExecution = launchJobOn(runningWorkerNode.get()); jobContext.initJobExecution(new FailSafeJobExecution<>(jobExecution)); LogicalTopologyEventListener nodeLeftEventListener = new OnNodeLeft(); @@ -143,7 +141,7 @@ JobExecution failSafeExecute() { return jobContext.failSafeJobExecution(); } - private JobExecution launchJobOn(ClusterNode runningWorkerNode) { + private JobExecution launchJobOn(ClusterNode runningWorkerNode) { if (runningWorkerNode.name().equals(topologyService.localMember().name())) { return computeComponent.executeLocally( jobContext.executionOptions(), jobContext.units(), jobContext.jobClassName(), cancellationToken, @@ -191,7 +189,7 @@ private void selectNewWorker() { LOG.info("Restarting the job {} on node {}.", jobContext.jobClassName(), nextWorker.name()); runningWorkerNode.set(nextWorker); - JobExecution jobExecution = launchJobOn(runningWorkerNode.get()); + JobExecution jobExecution = launchJobOn(runningWorkerNode.get()); jobContext.updateJobExecution(jobExecution); }); } diff --git a/modules/compute/src/main/java/org/apache/ignite/internal/compute/ComputeUtils.java b/modules/compute/src/main/java/org/apache/ignite/internal/compute/ComputeUtils.java index 3f0664d1a7e..33864c6f1d3 100644 --- a/modules/compute/src/main/java/org/apache/ignite/internal/compute/ComputeUtils.java +++ b/modules/compute/src/main/java/org/apache/ignite/internal/compute/ComputeUtils.java @@ -19,28 +19,15 @@ import static java.util.concurrent.CompletableFuture.completedFuture; import static java.util.concurrent.CompletableFuture.failedFuture; -import static org.apache.ignite.internal.compute.ComputeJobDataType.MARSHALLED_CUSTOM; -import static org.apache.ignite.internal.compute.ComputeJobDataType.NATIVE; -import static org.apache.ignite.internal.compute.ComputeJobDataType.POJO; -import static org.apache.ignite.internal.compute.ComputeJobDataType.TUPLE; -import static org.apache.ignite.internal.compute.ComputeJobDataType.TUPLE_COLLECTION; -import static org.apache.ignite.internal.compute.PojoConverter.fromTuple; -import static org.apache.ignite.internal.compute.PojoConverter.toTuple; import static org.apache.ignite.internal.util.ExceptionUtils.unwrapCause; import static org.apache.ignite.lang.ErrorGroups.Compute.CLASS_INITIALIZATION_ERR; import static org.apache.ignite.lang.ErrorGroups.Compute.COMPUTE_JOB_FAILED_ERR; import static org.apache.ignite.lang.ErrorGroups.Compute.MARSHALLING_TYPE_MISMATCH_ERR; import java.lang.reflect.Constructor; -import java.lang.reflect.InvocationTargetException; import java.lang.reflect.Method; -import java.nio.ByteBuffer; -import java.nio.ByteOrder; -import java.util.ArrayList; -import java.util.Arrays; import java.util.Collection; import java.util.List; -import java.util.Set; import java.util.UUID; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CompletionException; @@ -53,10 +40,6 @@ import org.apache.ignite.compute.task.TaskExecutionContext; import org.apache.ignite.deployment.DeploymentUnit; import org.apache.ignite.deployment.version.Version; -import org.apache.ignite.internal.binarytuple.BinaryTupleBuilder; -import org.apache.ignite.internal.binarytuple.BinaryTupleReader; -import org.apache.ignite.internal.binarytuple.inlineschema.TupleWithSchemaMarshalling; -import org.apache.ignite.internal.client.proto.ClientBinaryTupleUtils; import org.apache.ignite.internal.compute.loader.JobClassLoader; import org.apache.ignite.internal.compute.message.DeploymentUnitMsg; import org.apache.ignite.internal.compute.message.ExecuteResponse; @@ -65,13 +48,9 @@ import org.apache.ignite.internal.compute.message.JobResultResponse; import org.apache.ignite.internal.compute.message.JobStateResponse; import org.apache.ignite.internal.compute.message.JobStatesResponse; -import org.apache.ignite.internal.util.IgniteUtils; import org.apache.ignite.lang.IgniteCheckedException; import org.apache.ignite.lang.IgniteException; import org.apache.ignite.marshalling.Marshaller; -import org.apache.ignite.marshalling.MarshallingException; -import org.apache.ignite.marshalling.UnmarshallingException; -import org.apache.ignite.sql.ColumnType; import org.apache.ignite.table.DataStreamerReceiver; import org.apache.ignite.table.Tuple; import org.jetbrains.annotations.Nullable; @@ -82,10 +61,6 @@ public class ComputeUtils { private static final ComputeMessagesFactory MESSAGES_FACTORY = new ComputeMessagesFactory(); - private static final Set> NATIVE_TYPES = Arrays.stream(ColumnType.values()) - .map(ColumnType::javaClass) - .collect(Collectors.toUnmodifiableSet()); - /** * Instantiate compute job via provided class loader by provided job class. * @@ -255,16 +230,15 @@ public static CompletableFuture jobIdFromExecuteResponse(ExecuteResponse e * Extract Compute job result from execute response. * * @param jobResultResponse Job execution result message response. - * @param Compute job return type. * @return Completable future with result. */ - public static CompletableFuture resultFromJobResultResponse(JobResultResponse jobResultResponse) { + public static CompletableFuture resultFromJobResultResponse(JobResultResponse jobResultResponse) { Throwable throwable = jobResultResponse.throwable(); if (throwable != null) { return failedFuture(throwable); } - return completedFuture((R) jobResultResponse.result()); + return completedFuture(jobResultResponse.result()); } /** @@ -388,14 +362,14 @@ private static Throwable mapToComputeException(Throwable origin) { } if (input instanceof ComputeJobDataHolder) { - return unmarshalArgumentFromDataHolder(marshaller, (ComputeJobDataHolder) input, pojoType); + return SharedComputeUtils.unmarshalArgOrResult((ComputeJobDataHolder) input, marshaller, pojoType); } if (marshaller == null) { if (input instanceof Tuple) { // If input was marshalled as Tuple and argument type is not tuple then it's a pojo. if (pojoType != null && pojoType != Tuple.class) { - return (T) unmarshalPojo(pojoType, (Tuple) input); + return (T) SharedComputeUtils.unmarshalPojo(pojoType, (Tuple) input); } } return (T) input; @@ -419,185 +393,6 @@ private static Throwable mapToComputeException(Throwable origin) { ); } - /** - * Unmarshals the input from the {@link ComputeJobDataHolder} using provided marshaller if input was marshalled on the client. If the - * input was marshalled as a {@link Tuple} or POJO, then, if provided pojo type is not {@code null} and not a {@link Tuple}, unmarshals - * the input as a pojo using the provided pojo type, otherwise unmarshals it as a {@link Tuple}. - * - * @param marshaller Optional marshaller to unmarshal the input. - * @param argumentHolder Argument holder. - * @param pojoType Pojo type to use when unmarshalling as a pojo. - * @param Result type. - * @return Unmarshalled object. - */ - private static @Nullable T unmarshalArgumentFromDataHolder( - @Nullable Marshaller marshaller, - ComputeJobDataHolder argumentHolder, - @Nullable Class pojoType - ) { - ComputeJobDataType type = argumentHolder.type(); - if (type != MARSHALLED_CUSTOM && marshaller != null) { - throw new ComputeException( - MARSHALLING_TYPE_MISMATCH_ERR, - "Marshaller is defined on the server, but the argument was not marshalled on the client. " - + "If you want to use default marshalling strategy, " - + "then you should not define your marshaller in the job. " - + "If you would like to use your own marshaller, then double-check " - + "that both of them are defined in the client and in the server." - ); - } - switch (type) { - case NATIVE: { - var reader = new BinaryTupleReader(3, argumentHolder.data()); - return (T) ClientBinaryTupleUtils.readObject(reader, 0); - } - - case TUPLE: // Fallthrough TODO https://issues.apache.org/jira/browse/IGNITE-23320 - case POJO: - Tuple tuple = TupleWithSchemaMarshalling.unmarshal(argumentHolder.data()); - if (pojoType != null && pojoType != Tuple.class) { - return (T) unmarshalPojo(pojoType, tuple); - } - return (T) tuple; - - case MARSHALLED_CUSTOM: - if (marshaller == null) { - throw new ComputeException(MARSHALLING_TYPE_MISMATCH_ERR, "Marshaller should be defined on the client"); - } - try { - return marshaller.unmarshal(argumentHolder.data()); - } catch (Exception ex) { - throw new ComputeException(MARSHALLING_TYPE_MISMATCH_ERR, "Exception in user-defined marshaller", ex); - } - - case TUPLE_COLLECTION: { - // TODO: IGNITE-24059 Deduplicate with ClientComputeJobUnpacker. - ByteBuffer collectionBuf = ByteBuffer.wrap(argumentHolder.data()).order(ByteOrder.LITTLE_ENDIAN); - int count = collectionBuf.getInt(); - BinaryTupleReader reader = new BinaryTupleReader(count, collectionBuf.slice().order(ByteOrder.LITTLE_ENDIAN)); - - List res = new ArrayList<>(count); - for (int i = 0; i < count; i++) { - ByteBuffer elementBytes = reader.bytesValueAsBuffer(i); - - if (elementBytes == null) { - res.add(null); - continue; - } - - res.add(TupleWithSchemaMarshalling.unmarshal(elementBytes)); - } - - return (T) res; - } - - default: - throw new ComputeException(MARSHALLING_TYPE_MISMATCH_ERR, "Unexpected job argument type: " + type); - } - } - - private static Object unmarshalPojo(Class pojoType, Tuple input) { - try { - Object obj = pojoType.getConstructor().newInstance(); - - fromTuple(obj, input); - - return obj; - } catch (NoSuchMethodException e) { - throw new UnmarshallingException("Class " + pojoType.getName() + " doesn't have public default constructor. " - + "Add the constructor or define argument marshaller in the compute job.", e); - } catch (InvocationTargetException e) { - throw new UnmarshallingException("Constructor has thrown an exception", e); - } catch (InstantiationException e) { - throw new UnmarshallingException("Can't instantiate an object of class " + pojoType.getName(), e); - } catch (IllegalAccessException e) { - throw new UnmarshallingException("Constructor is inaccessible", e); - } catch (PojoConversionException e) { - throw new UnmarshallingException("Can't unpack object", e); - } - } - - /** - * Marshals the job result using either provided marshaller if not {@code null} or depending on the type of the result either as a - * {@link Tuple}, a native type (see {@link ColumnType}) or a POJO. Wraps the marshalled data with the data type in the - * {@link ComputeJobDataHolder} to be unmarshalled on the client. - * - * @param result Compute job result. - * @param marshaller Optional result marshaller. - * - * @return Data holder. - */ - @Nullable - static ComputeJobDataHolder marshalAndWrapResult(Object result, @Nullable Marshaller marshaller) { - if (result == null) { - return null; - } - - if (marshaller != null) { - byte[] data = marshaller.marshal(result); - if (data == null) { - return null; - } - return new ComputeJobDataHolder(MARSHALLED_CUSTOM, data); - } - - if (result instanceof Tuple) { - Tuple tuple = (Tuple) result; - return new ComputeJobDataHolder(TUPLE, TupleWithSchemaMarshalling.marshal(tuple)); - } - - if (result instanceof Collection) { - // TODO: IGNITE-24059 Deduplicate with ClientComputeJobPacker. - Collection col = (Collection) result; - - // Pack entire collection into a single binary blob. - BinaryTupleBuilder tupleBuilder = new BinaryTupleBuilder(col.size()); - - for (Object el : col) { - if (el == null) { - tupleBuilder.appendNull(); - continue; - } - - if (!(el instanceof Tuple)) { - throw new MarshallingException("Can't pack collection: expected Tuple, but got " + el.getClass(), null); - } - - tupleBuilder.appendBytes(TupleWithSchemaMarshalling.marshal((Tuple) el)); - } - - ByteBuffer binTupleBytes = tupleBuilder.build(); - - byte[] resArr = new byte[Integer.BYTES + binTupleBytes.remaining()]; - ByteBuffer resBuf = ByteBuffer.wrap(resArr).order(ByteOrder.LITTLE_ENDIAN); - resBuf.putInt(col.size()); - resBuf.put(binTupleBytes); - - return new ComputeJobDataHolder(TUPLE_COLLECTION, resArr); - } - - - if (isNativeType(result.getClass())) { - // Builder with inline schema. - // Value is represented by 3 tuple elements: type, scale, value. - var builder = new BinaryTupleBuilder(3, 3, false); - ClientBinaryTupleUtils.appendObject(builder, result); - return new ComputeJobDataHolder(NATIVE, IgniteUtils.byteBufferToByteArray(builder.build())); - } - - try { - // TODO https://issues.apache.org/jira/browse/IGNITE-23320 - Tuple tuple = toTuple(result); - return new ComputeJobDataHolder(POJO, TupleWithSchemaMarshalling.marshal(tuple)); - } catch (PojoConversionException e) { - throw new MarshallingException("Can't pack object", e); - } - } - - private static boolean isNativeType(Class clazz) { - return NATIVE_TYPES.contains(clazz); - } - /** * Finds the second argument type of the {@link ComputeJob#executeAsync(JobExecutionContext, T)} method in the provided job class. * diff --git a/modules/compute/src/main/java/org/apache/ignite/internal/compute/DelegatingJobExecution.java b/modules/compute/src/main/java/org/apache/ignite/internal/compute/DelegatingJobExecution.java index de376d8be27..00c4d18cd6c 100644 --- a/modules/compute/src/main/java/org/apache/ignite/internal/compute/DelegatingJobExecution.java +++ b/modules/compute/src/main/java/org/apache/ignite/internal/compute/DelegatingJobExecution.java @@ -18,27 +18,23 @@ package org.apache.ignite.internal.compute; import java.util.concurrent.CompletableFuture; -import java.util.concurrent.ExecutionException; import org.apache.ignite.compute.JobExecution; import org.apache.ignite.compute.JobState; import org.apache.ignite.internal.compute.executor.JobExecutionInternal; -import org.apache.ignite.marshalling.Marshaller; import org.jetbrains.annotations.Nullable; /** * Delegates {@link JobExecution} to the future of {@link JobExecutionInternal}. - * - * @param Result type. */ -class DelegatingJobExecution implements JobExecution, MarshallerProvider { - private final CompletableFuture> delegate; +class DelegatingJobExecution implements JobExecution { + private final CompletableFuture> delegate; - DelegatingJobExecution(CompletableFuture> delegate) { + DelegatingJobExecution(CompletableFuture> delegate) { this.delegate = delegate; } @Override - public CompletableFuture resultAsync() { + public CompletableFuture resultAsync() { return delegate.thenCompose(JobExecutionInternal::resultAsync); } @@ -56,22 +52,4 @@ public CompletableFuture resultAsync() { public CompletableFuture<@Nullable Boolean> changePriorityAsync(int newPriority) { return delegate.thenApply(jobExecutionInternal -> jobExecutionInternal.changePriority(newPriority)); } - - @Override - public @Nullable Marshaller resultMarshaller() { - try { - return delegate.thenApply(JobExecutionInternal::resultMarshaller).get(); - } catch (InterruptedException | ExecutionException e) { - throw new RuntimeException(e); - } - } - - @Override - public boolean marshalResult() { - try { - return delegate.thenApply(JobExecutionInternal::marshalResult).get(); - } catch (InterruptedException | ExecutionException e) { - throw new RuntimeException(e); - } - } } diff --git a/modules/compute/src/main/java/org/apache/ignite/internal/compute/ExecutionManager.java b/modules/compute/src/main/java/org/apache/ignite/internal/compute/ExecutionManager.java index cf4101a675e..9287a7b7aab 100644 --- a/modules/compute/src/main/java/org/apache/ignite/internal/compute/ExecutionManager.java +++ b/modules/compute/src/main/java/org/apache/ignite/internal/compute/ExecutionManager.java @@ -36,7 +36,6 @@ import org.apache.ignite.internal.compute.configuration.ComputeConfiguration; import org.apache.ignite.internal.compute.messaging.RemoteJobExecution; import org.apache.ignite.internal.network.TopologyService; -import org.apache.ignite.marshalling.Marshaller; import org.jetbrains.annotations.Nullable; import org.jetbrains.annotations.TestOnly; @@ -93,22 +92,9 @@ void stop() { public CompletableFuture resultAsync(UUID jobId) { JobExecution execution = executions.get(jobId); if (execution != null) { - if (execution instanceof MarshallerProvider) { - MarshallerProvider provider = (MarshallerProvider) execution; - Marshaller marshaller = provider.resultMarshaller(); - - // If result needs to be marshalled, then job execution request came from the client and we need to marshal the result and - // return the wrapper object back to the client handler node so it can pass the binary data directly back to client. - if (provider.marshalResult()) { - return execution.resultAsync().thenApply(result -> ComputeUtils.marshalAndWrapResult(result, marshaller)); - } - - if (marshaller != null) { - return execution.resultAsync().thenApply(marshaller::marshal); - } - } return execution.resultAsync(); } + return failedFuture(new ComputeException(RESULT_NOT_FOUND_ERR, "Job result not found for the job with ID: " + jobId)); } diff --git a/modules/compute/src/main/java/org/apache/ignite/internal/compute/IgniteComputeImpl.java b/modules/compute/src/main/java/org/apache/ignite/internal/compute/IgniteComputeImpl.java index ddc4b5b40c8..04e7daca294 100644 --- a/modules/compute/src/main/java/org/apache/ignite/internal/compute/IgniteComputeImpl.java +++ b/modules/compute/src/main/java/org/apache/ignite/internal/compute/IgniteComputeImpl.java @@ -24,7 +24,6 @@ import static org.apache.ignite.internal.util.CompletableFutures.nullCompletedFuture; import static org.apache.ignite.internal.util.ExceptionUtils.unwrapCause; import static org.apache.ignite.lang.ErrorGroups.Compute.COMPUTE_JOB_FAILED_ERR; -import static org.apache.ignite.marshalling.Marshaller.tryMarshalOrCast; import java.util.Collection; import java.util.HashSet; @@ -126,9 +125,10 @@ JobExecution submit(JobTarget target, JobDescriptor descriptor, return new ResultUnmarshallingJobExecution<>( executeAsyncWithFailover( nodes, descriptor.units(), descriptor.jobClassName(), descriptor.options(), cancellationToken, - tryMarshalOrCast(argumentMarshaller, args) + SharedComputeUtils.marshalArgOrResult(args, argumentMarshaller) ), - resultMarshaller + resultMarshaller, + descriptor.resultClass() ); } @@ -138,7 +138,7 @@ JobExecution submit(JobTarget target, JobDescriptor descriptor, String tableName = colocatedTarget.tableName(); Object key = colocatedTarget.key(); - CompletableFuture> jobFut; + CompletableFuture> jobFut; if (mapper != null) { jobFut = requiredTable(tableName) .thenCompose(table -> primaryReplicaForPartitionByMappedKey(table, key, mapper) @@ -148,7 +148,7 @@ JobExecution submit(JobTarget target, JobDescriptor descriptor, descriptor.units(), descriptor.jobClassName(), descriptor.options(), cancellationToken, - tryMarshalOrCast(argumentMarshaller, args) + SharedComputeUtils.marshalArgOrResult(args, argumentMarshaller) ))); } else { @@ -160,11 +160,11 @@ JobExecution submit(JobTarget target, JobDescriptor descriptor, descriptor.jobClassName(), descriptor.options(), cancellationToken, - tryMarshalOrCast(argumentMarshaller, args))) - .thenApply(job -> (JobExecution) job); + SharedComputeUtils.marshalArgOrResult(args, argumentMarshaller))); } - return new ResultUnmarshallingJobExecution<>(new JobExecutionFutureWrapper<>(jobFut), resultMarshaller); + return new ResultUnmarshallingJobExecution<>( + new JobExecutionFutureWrapper<>(jobFut), resultMarshaller, descriptor.resultClass()); } throw new IllegalArgumentException("Unsupported job target: " + target); @@ -188,13 +188,13 @@ public R execute(JobTarget target, JobDescriptor descriptor, @Nulla } @Override - public JobExecution executeAsyncWithFailover( + public JobExecution executeAsyncWithFailover( Set nodes, List units, String jobClassName, JobExecutionOptions options, @Nullable CancellationToken cancellationToken, - @Nullable Object arg + @Nullable ComputeJobDataHolder arg ) { Set candidates = new HashSet<>(); for (ClusterNode node : nodes) { @@ -235,14 +235,14 @@ private static ClusterNode randomNode(Set nodes) { return iterator.next(); } - private JobExecution executeOnOneNodeWithFailover( + private JobExecution executeOnOneNodeWithFailover( ClusterNode targetNode, NextWorkerSelector nextWorkerSelector, List units, String jobClassName, JobExecutionOptions jobExecutionOptions, @Nullable CancellationToken cancellationToken, - @Nullable T arg + @Nullable ComputeJobDataHolder arg ) { ExecutionOptions options = ExecutionOptions.from(jobExecutionOptions); @@ -276,14 +276,14 @@ private boolean isLocal(ClusterNode targetNode) { } @Override - public CompletableFuture> submitColocatedInternal( + public CompletableFuture> submitColocatedInternal( TableViewInternal table, Tuple key, List units, String jobClassName, JobExecutionOptions options, @Nullable CancellationToken cancellationToken, - @Nullable Object arg) { + @Nullable ComputeJobDataHolder arg) { return primaryReplicaForPartitionByTupleKey(table, key) .thenApply(primaryNode -> executeOnOneNodeWithFailover( primaryNode, @@ -340,6 +340,7 @@ public Map> submitBroadcast( Marshaller argumentMarshaller = descriptor.argumentMarshaller(); Marshaller resultMarshaller = descriptor.resultMarshaller(); + ComputeJobDataHolder argHolder = SharedComputeUtils.marshalArgOrResult(args, argumentMarshaller); return nodes.stream() .collect(toUnmodifiableMap(identity(), @@ -349,13 +350,19 @@ public Map> submitBroadcast( if (topologyService.getByConsistentId(node.name()) == null) { return new FailedExecution<>(new NodeNotFoundException(Set.of(node.name()))); } + return new ResultUnmarshallingJobExecution<>( new JobExecutionWrapper<>( executeOnOneNodeWithFailover( - node, CompletableFutures::nullCompletedFuture, - descriptor.units(), descriptor.jobClassName(), - descriptor.options(), null, tryMarshalOrCast(argumentMarshaller, args))), - resultMarshaller); + node, + CompletableFutures::nullCompletedFuture, + descriptor.units(), + descriptor.jobClassName(), + descriptor.options(), + null, + argHolder)), + resultMarshaller, + descriptor.resultClass()); })); } @@ -429,13 +436,13 @@ public CompletableFuture> runReceiverAsync( @Override public CompletableFuture runReceiverAsync(byte[] payload, ClusterNode node, List deploymentUnits) { // Use Compute to execute receiver on the target node with failover, class loading, scheduling. - JobExecution jobExecution = executeAsyncWithFailover( + JobExecution jobExecution = executeAsyncWithFailover( Set.of(node), deploymentUnits, StreamerReceiverJob.class.getName(), JobExecutionOptions.DEFAULT, null, - payload); + SharedComputeUtils.marshalArgOrResult(payload, null)); return jobExecution.resultAsync() .handle((res, err) -> { @@ -450,7 +457,7 @@ public CompletableFuture runReceiverAsync(byte[] payload, ClusterNode no ExceptionUtils.sneakyThrow(err); } - return res; + return SharedComputeUtils.unmarshalArgOrResult(res, null, null); }); } diff --git a/modules/compute/src/main/java/org/apache/ignite/internal/compute/IgniteComputeInternal.java b/modules/compute/src/main/java/org/apache/ignite/internal/compute/IgniteComputeInternal.java index 90835bc4f4a..db3453e2db3 100644 --- a/modules/compute/src/main/java/org/apache/ignite/internal/compute/IgniteComputeInternal.java +++ b/modules/compute/src/main/java/org/apache/ignite/internal/compute/IgniteComputeInternal.java @@ -42,7 +42,6 @@ public interface IgniteComputeInternal extends IgniteCompute { * Executes a {@link ComputeJob} of the given class on a single node. If the node leaves the cluster, it will be restarted on one of the * candidate nodes. * - * @param Job result type. * @param nodes Candidate nodes; In case target node left the cluster, the job will be restarted on one of them. * @param units Deployment units. Can be empty. * @param jobClassName Name of the job class to execute. @@ -51,13 +50,13 @@ public interface IgniteComputeInternal extends IgniteCompute { * @param payload Arguments of the job. * @return CompletableFuture Job result. */ - JobExecution executeAsyncWithFailover( + JobExecution executeAsyncWithFailover( Set nodes, List units, String jobClassName, JobExecutionOptions options, @Nullable CancellationToken cancellationToken, - @Nullable Object payload + @Nullable ComputeJobDataHolder payload ); /** @@ -71,17 +70,16 @@ JobExecution executeAsyncWithFailover( * @param options job execution options (priority, max retries). * @param cancellationToken Cancellation token or {@code null}. * @param payload Arguments of the job. - * @param Job result type. * @return Job execution object. */ - CompletableFuture> submitColocatedInternal( + CompletableFuture> submitColocatedInternal( TableViewInternal table, Tuple key, List units, String jobClassName, JobExecutionOptions options, @Nullable CancellationToken cancellationToken, - Object payload); + @Nullable ComputeJobDataHolder payload); /** * Wraps the given future into a job execution object. diff --git a/modules/compute/src/main/java/org/apache/ignite/internal/compute/JobStarter.java b/modules/compute/src/main/java/org/apache/ignite/internal/compute/JobStarter.java index 941a4ad263c..a561d0ea8dd 100644 --- a/modules/compute/src/main/java/org/apache/ignite/internal/compute/JobStarter.java +++ b/modules/compute/src/main/java/org/apache/ignite/internal/compute/JobStarter.java @@ -20,6 +20,7 @@ import java.util.List; import org.apache.ignite.compute.JobExecution; import org.apache.ignite.deployment.DeploymentUnit; +import org.jetbrains.annotations.Nullable; /** * Compute job starter interface. @@ -32,13 +33,12 @@ public interface JobStarter { * @param units Deployment units. Can be empty. * @param jobClassName Name of the job class to execute. * @param args Arguments of the job. - * @param Job result type. * @return CompletableFuture Job result. */ - JobExecution start( + JobExecution start( ExecutionOptions options, List units, String jobClassName, - Object args + @Nullable ComputeJobDataHolder args ); } diff --git a/modules/compute/src/main/java/org/apache/ignite/internal/compute/RemoteExecutionContext.java b/modules/compute/src/main/java/org/apache/ignite/internal/compute/RemoteExecutionContext.java index 4aefcfc303f..bab48411d4b 100644 --- a/modules/compute/src/main/java/org/apache/ignite/internal/compute/RemoteExecutionContext.java +++ b/modules/compute/src/main/java/org/apache/ignite/internal/compute/RemoteExecutionContext.java @@ -25,23 +25,19 @@ /** * Captures the context of a remote job execution. Also provides methods to access the job execution object * that is returned to the user. The access is thread safe. - * - * @param type of the input of the job. - * @param type of the result of the job. */ -class RemoteExecutionContext { - +class RemoteExecutionContext { private final ExecutionOptions executionOptions; private final List units; private final String jobClassName; - private final T arg; + private final ComputeJobDataHolder arg; - private final AtomicReference> jobExecution; + private final AtomicReference> jobExecution; - RemoteExecutionContext(List units, String jobClassName, ExecutionOptions executionOptions, T arg) { + RemoteExecutionContext(List units, String jobClassName, ExecutionOptions executionOptions, ComputeJobDataHolder arg) { this.executionOptions = executionOptions; this.units = units; this.jobClassName = jobClassName; @@ -54,7 +50,7 @@ class RemoteExecutionContext { * * @param jobExecution the instance of job execution that should be returned to the client. */ - void initJobExecution(FailSafeJobExecution jobExecution) { + void initJobExecution(FailSafeJobExecution jobExecution) { if (!this.jobExecution.compareAndSet(null, jobExecution)) { throw new IllegalStateException("Job execution is already initialized."); } @@ -65,8 +61,8 @@ void initJobExecution(FailSafeJobExecution jobExecution) { * * @return fail-safe job execution object. */ - FailSafeJobExecution failSafeJobExecution() { - FailSafeJobExecution jobExecution = this.jobExecution.get(); + FailSafeJobExecution failSafeJobExecution() { + FailSafeJobExecution jobExecution = this.jobExecution.get(); if (jobExecution == null) { throw new IllegalStateException("Job execution is not initialized. Call initJobExecution() first."); } @@ -80,7 +76,7 @@ FailSafeJobExecution failSafeJobExecution() { * * @param jobExecution the new job execution object (supposed to be a restarted job but in another worker node). */ - void updateJobExecution(JobExecution jobExecution) { + void updateJobExecution(JobExecution jobExecution) { failSafeJobExecution().updateJobExecution(jobExecution); } @@ -96,7 +92,7 @@ String jobClassName() { return jobClassName; } - T arg() { + ComputeJobDataHolder arg() { return arg; } } diff --git a/modules/compute/src/main/java/org/apache/ignite/internal/compute/ResultUnmarshallingJobExecution.java b/modules/compute/src/main/java/org/apache/ignite/internal/compute/ResultUnmarshallingJobExecution.java index b6145258a02..54f8099b9cd 100644 --- a/modules/compute/src/main/java/org/apache/ignite/internal/compute/ResultUnmarshallingJobExecution.java +++ b/modules/compute/src/main/java/org/apache/ignite/internal/compute/ResultUnmarshallingJobExecution.java @@ -17,8 +17,6 @@ package org.apache.ignite.internal.compute; -import static org.apache.ignite.marshalling.Marshaller.tryUnmarshalOrCast; - import java.util.concurrent.CompletableFuture; import org.apache.ignite.compute.JobExecution; import org.apache.ignite.compute.JobState; @@ -31,17 +29,23 @@ * @param Result type. */ class ResultUnmarshallingJobExecution implements JobExecution { - private final JobExecution delegate; - private final Marshaller resultUnmarshaller; + private final JobExecution delegate; + private final @Nullable Marshaller resultUnmarshaller; + private final @Nullable Class resultClass; - ResultUnmarshallingJobExecution(JobExecution delegate, @Nullable Marshaller resultUnmarshaller) { + ResultUnmarshallingJobExecution( + JobExecution delegate, + @Nullable Marshaller resultUnmarshaller, + @Nullable Class resultClass) { this.delegate = delegate; this.resultUnmarshaller = resultUnmarshaller; + this.resultClass = resultClass; } @Override public CompletableFuture resultAsync() { - return delegate.resultAsync().thenApply(r -> tryUnmarshalOrCast(resultUnmarshaller, r)); + return delegate.resultAsync().thenApply( + r -> SharedComputeUtils.unmarshalArgOrResult(r, resultUnmarshaller, resultClass)); } @Override diff --git a/modules/compute/src/main/java/org/apache/ignite/internal/compute/executor/ComputeExecutor.java b/modules/compute/src/main/java/org/apache/ignite/internal/compute/executor/ComputeExecutor.java index 9588ccaa607..08632bf1919 100644 --- a/modules/compute/src/main/java/org/apache/ignite/internal/compute/executor/ComputeExecutor.java +++ b/modules/compute/src/main/java/org/apache/ignite/internal/compute/executor/ComputeExecutor.java @@ -19,20 +19,22 @@ import org.apache.ignite.compute.ComputeJob; import org.apache.ignite.compute.task.MapReduceTask; +import org.apache.ignite.internal.compute.ComputeJobDataHolder; import org.apache.ignite.internal.compute.ExecutionOptions; import org.apache.ignite.internal.compute.loader.JobClassLoader; import org.apache.ignite.internal.compute.task.JobSubmitter; import org.apache.ignite.internal.compute.task.TaskExecutionInternal; +import org.jetbrains.annotations.Nullable; /** * Executor of Compute jobs. */ public interface ComputeExecutor { - JobExecutionInternal executeJob( + JobExecutionInternal executeJob( ExecutionOptions options, Class> jobClass, JobClassLoader classLoader, - T input); + @Nullable ComputeJobDataHolder input); TaskExecutionInternal executeTask( JobSubmitter jobSubmitter, diff --git a/modules/compute/src/main/java/org/apache/ignite/internal/compute/executor/ComputeExecutorImpl.java b/modules/compute/src/main/java/org/apache/ignite/internal/compute/executor/ComputeExecutorImpl.java index d696f2d2b77..06e99b80a55 100644 --- a/modules/compute/src/main/java/org/apache/ignite/internal/compute/executor/ComputeExecutorImpl.java +++ b/modules/compute/src/main/java/org/apache/ignite/internal/compute/executor/ComputeExecutorImpl.java @@ -34,6 +34,7 @@ import org.apache.ignite.internal.compute.ComputeUtils; import org.apache.ignite.internal.compute.ExecutionOptions; import org.apache.ignite.internal.compute.JobExecutionContextImpl; +import org.apache.ignite.internal.compute.SharedComputeUtils; import org.apache.ignite.internal.compute.configuration.ComputeConfiguration; import org.apache.ignite.internal.compute.loader.JobClassLoader; import org.apache.ignite.internal.compute.queue.PriorityQueueExecutor; @@ -80,11 +81,11 @@ public ComputeExecutorImpl( } @Override - public JobExecutionInternal executeJob( + public JobExecutionInternal executeJob( ExecutionOptions options, Class> jobClass, JobClassLoader classLoader, - T input + ComputeJobDataHolder input ) { assert executorService != null; @@ -94,27 +95,31 @@ public JobExecutionInternal executeJob( Marshaller inputMarshaller = jobInstance.inputMarshaller(); Marshaller resultMarshaller = jobInstance.resultMarshaller(); - // If input is of this type, this means that the request came from the thin client and packing the result to the byte array will be - // needed in any case. In order to minimize conversion, marshal the result here. - boolean marshalResult = input instanceof ComputeJobDataHolder; - - QueueExecution execution = executorService.submit( - unmarshalExecMarshal(input, jobClass, jobInstance, context, inputMarshaller), + QueueExecution execution = executorService.submit( + unmarshalExecMarshal(input, jobClass, jobInstance, context, inputMarshaller, resultMarshaller), options.priority(), options.maxRetries() ); - return new JobExecutionInternal<>(execution, isInterrupted, resultMarshaller, marshalResult); + return new JobExecutionInternal<>(execution, isInterrupted, null, false); } - private static Callable> unmarshalExecMarshal( - T input, + private static Callable> unmarshalExecMarshal( + ComputeJobDataHolder input, Class> jobClass, ComputeJob jobInstance, JobExecutionContext context, - @Nullable Marshaller inputMarshaller + @Nullable Marshaller inputMarshaller, + @Nullable Marshaller resultMarshaller ) { - return () -> jobInstance.executeAsync(context, unmarshalOrNotIfNull(inputMarshaller, input, getJobExecuteArgumentType(jobClass))); + return () -> { + CompletableFuture userJobFut = jobInstance.executeAsync( + context, unmarshalOrNotIfNull(inputMarshaller, input, getJobExecuteArgumentType(jobClass))); + + return userJobFut == null + ? null + : userJobFut.thenApply(r -> SharedComputeUtils.marshalArgOrResult(r, resultMarshaller)); + }; } @Override diff --git a/modules/compute/src/main/java/org/apache/ignite/internal/compute/executor/JobExecutionInternal.java b/modules/compute/src/main/java/org/apache/ignite/internal/compute/executor/JobExecutionInternal.java index 093e11b0627..84953fcd840 100644 --- a/modules/compute/src/main/java/org/apache/ignite/internal/compute/executor/JobExecutionInternal.java +++ b/modules/compute/src/main/java/org/apache/ignite/internal/compute/executor/JobExecutionInternal.java @@ -20,6 +20,7 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.atomic.AtomicBoolean; import org.apache.ignite.compute.JobState; +import org.apache.ignite.internal.compute.ComputeJobDataHolder; import org.apache.ignite.internal.compute.MarshallerProvider; import org.apache.ignite.internal.compute.queue.QueueExecution; import org.apache.ignite.marshalling.Marshaller; @@ -31,7 +32,7 @@ * @param Job result type. */ public class JobExecutionInternal implements MarshallerProvider { - private final QueueExecution execution; + private final QueueExecution execution; private final AtomicBoolean isInterrupted; @@ -48,7 +49,7 @@ public class JobExecutionInternal implements MarshallerProvider { * @param marshalResult Flag indicating whether the marshalling of the result will be needed. */ JobExecutionInternal( - QueueExecution execution, + QueueExecution execution, AtomicBoolean isInterrupted, @Nullable Marshaller marshaller, boolean marshalResult @@ -59,7 +60,7 @@ public class JobExecutionInternal implements MarshallerProvider { this.marshalResult = marshalResult; } - public CompletableFuture resultAsync() { + public CompletableFuture resultAsync() { return execution.resultAsync(); } diff --git a/modules/compute/src/main/java/org/apache/ignite/internal/compute/message/ExecuteRequest.java b/modules/compute/src/main/java/org/apache/ignite/internal/compute/message/ExecuteRequest.java index e82dcfcdb76..f367e73a826 100644 --- a/modules/compute/src/main/java/org/apache/ignite/internal/compute/message/ExecuteRequest.java +++ b/modules/compute/src/main/java/org/apache/ignite/internal/compute/message/ExecuteRequest.java @@ -20,6 +20,7 @@ import java.util.List; import java.util.Set; import org.apache.ignite.compute.JobDescriptor; +import org.apache.ignite.internal.compute.ComputeJobDataHolder; import org.apache.ignite.internal.compute.ComputeMessageTypes; import org.apache.ignite.internal.compute.ExecutionOptions; import org.apache.ignite.internal.network.NetworkMessage; @@ -61,5 +62,5 @@ public interface ExecuteRequest extends NetworkMessage { */ @Marshallable @Nullable - Object input(); + ComputeJobDataHolder input(); } diff --git a/modules/compute/src/main/java/org/apache/ignite/internal/compute/message/JobResultResponse.java b/modules/compute/src/main/java/org/apache/ignite/internal/compute/message/JobResultResponse.java index afed9e3f387..d9b54ce3677 100644 --- a/modules/compute/src/main/java/org/apache/ignite/internal/compute/message/JobResultResponse.java +++ b/modules/compute/src/main/java/org/apache/ignite/internal/compute/message/JobResultResponse.java @@ -17,6 +17,7 @@ package org.apache.ignite.internal.compute.message; +import org.apache.ignite.internal.compute.ComputeJobDataHolder; import org.apache.ignite.internal.compute.ComputeMessageTypes; import org.apache.ignite.internal.network.NetworkMessage; import org.apache.ignite.internal.network.annotations.Marshallable; @@ -35,7 +36,7 @@ public interface JobResultResponse extends NetworkMessage { */ @Nullable @Marshallable - Object result(); + ComputeJobDataHolder result(); /** * Returns a {@link Throwable} that was thrown during job execution ({@code null} if the execution was successful). diff --git a/modules/compute/src/main/java/org/apache/ignite/internal/compute/messaging/ComputeMessaging.java b/modules/compute/src/main/java/org/apache/ignite/internal/compute/messaging/ComputeMessaging.java index f5269021c7d..8b87c38600f 100644 --- a/modules/compute/src/main/java/org/apache/ignite/internal/compute/messaging/ComputeMessaging.java +++ b/modules/compute/src/main/java/org/apache/ignite/internal/compute/messaging/ComputeMessaging.java @@ -43,6 +43,7 @@ import org.apache.ignite.compute.JobExecution; import org.apache.ignite.compute.JobState; import org.apache.ignite.deployment.DeploymentUnit; +import org.apache.ignite.internal.compute.ComputeJobDataHolder; import org.apache.ignite.internal.compute.ComputeMessageTypes; import org.apache.ignite.internal.compute.ComputeMessagesFactory; import org.apache.ignite.internal.compute.ComputeUtils; @@ -174,12 +175,12 @@ public void stop() { * @param input Arguments of the job. * @return Job id future that will be completed when the job is submitted on the remote node. */ - public CompletableFuture remoteExecuteRequestAsync( + public CompletableFuture remoteExecuteRequestAsync( ExecutionOptions options, ClusterNode remoteNode, List units, String jobClassName, - T input + ComputeJobDataHolder input ) { List deploymentUnitMsgs = units.stream() .map(ComputeUtils::toDeploymentUnitMsg) @@ -199,7 +200,7 @@ public CompletableFuture remoteExecuteRequestAsync( private void processExecuteRequest(JobStarter starter, ExecuteRequest request, ClusterNode sender, long correlationId) { List units = toDeploymentUnit(request.deploymentUnits()); - JobExecution execution = starter.start( + JobExecution execution = starter.start( request.executeOptions(), units, request.jobClassName(), request.input() ); execution.idAsync().whenComplete((jobId, err) -> sendExecuteResponse(jobId, err, sender, correlationId)); @@ -219,10 +220,9 @@ private void sendExecuteResponse(@Nullable UUID jobId, @Nullable Throwable ex, C * * @param remoteNode The job will be executed on this node. * @param jobId Job id. - * @param Job result type * @return Job result. */ - public CompletableFuture remoteJobResultRequestAsync(ClusterNode remoteNode, UUID jobId) { + public CompletableFuture remoteJobResultRequestAsync(ClusterNode remoteNode, UUID jobId) { JobResultRequest jobResultRequest = messagesFactory.jobResultRequest() .jobId(jobId) .build(); @@ -233,10 +233,14 @@ public CompletableFuture remoteJobResultRequestAsync(ClusterNode remoteNo private void processJobResultRequest(JobResultRequest request, ClusterNode sender, long correlationId) { executionManager.resultAsync(request.jobId()) - .whenComplete((result, err) -> sendJobResultResponse(result, err, sender, correlationId)); + .whenComplete((result, err) -> sendJobResultResponse((ComputeJobDataHolder) result, err, sender, correlationId)); } - private void sendJobResultResponse(@Nullable Object result, @Nullable Throwable ex, ClusterNode sender, long correlationId) { + private void sendJobResultResponse( + @Nullable ComputeJobDataHolder result, + @Nullable Throwable ex, + ClusterNode sender, + long correlationId) { JobResultResponse jobResultResponse = messagesFactory.jobResultResponse() .result(result) .throwable(ex) diff --git a/modules/compute/src/test/java/org/apache/ignite/internal/compute/ComputeComponentImplTest.java b/modules/compute/src/test/java/org/apache/ignite/internal/compute/ComputeComponentImplTest.java index 707e36336fe..08bbe54d854 100644 --- a/modules/compute/src/test/java/org/apache/ignite/internal/compute/ComputeComponentImplTest.java +++ b/modules/compute/src/test/java/org/apache/ignite/internal/compute/ComputeComponentImplTest.java @@ -189,9 +189,10 @@ void cleanup() { @Test void executesLocally() { - JobExecution execution = computeComponent.executeLocally(DEFAULT, List.of(), SimpleJob.class.getName(), null, ""); + JobExecution execution = computeComponent.executeLocally( + DEFAULT, List.of(), SimpleJob.class.getName(), null, SharedComputeUtils.marshalArgOrResult("", null)); - assertThat(execution.resultAsync(), willBe("jobResponse")); + assertThat(unwrapResult(execution), willBe("jobResponse")); assertThat(execution.stateAsync(), willBe(jobStateWithStatus(COMPLETED))); assertThat(execution.cancelAsync(), willBe(false)); assertThat(execution.changePriorityAsync(1), willBe(false)); @@ -208,8 +209,8 @@ void testLongPreExecutionInitialization() { CancelHandle cancelHandle = CancelHandle.create(); - JobExecution execution = computeComponent.executeLocally(DEFAULT, List.of(), SimpleJob.class.getName(), - cancelHandle.token(), ""); + JobExecution execution = computeComponent.executeLocally(DEFAULT, List.of(), SimpleJob.class.getName(), + cancelHandle.token(), SharedComputeUtils.marshalArgOrResult("", null)); assertFalse(infiniteFuture.isDone()); assertFalse(execution.resultAsync().isDone()); @@ -224,7 +225,8 @@ void testLongPreExecutionInitialization() { @Test void getsStateAndCancelsLocally() { - JobExecution execution = computeComponent.executeLocally(DEFAULT, List.of(), LongJob.class.getName(), null, null); + JobExecution execution = computeComponent.executeLocally( + DEFAULT, List.of(), LongJob.class.getName(), null, null); await().until(execution::stateAsync, willBe(jobStateWithStatus(EXECUTING))); @@ -237,10 +239,12 @@ void getsStateAndCancelsLocally() { @Test void stateCancelAndChangePriorityTriesLocalNodeFirst() { - JobExecution runningExecution = computeComponent.executeLocally(DEFAULT, List.of(), LongJob.class.getName(), null, null); + JobExecution runningExecution = computeComponent.executeLocally( + DEFAULT, List.of(), LongJob.class.getName(), null, null); await().until(runningExecution::stateAsync, willBe(jobStateWithStatus(EXECUTING))); - JobExecution queuedExecution = computeComponent.executeLocally(DEFAULT, List.of(), LongJob.class.getName(), null, null); + JobExecution queuedExecution = computeComponent.executeLocally( + DEFAULT, List.of(), LongJob.class.getName(), null, null); await().until(queuedExecution::stateAsync, willBe(jobStateWithStatus(QUEUED))); UUID jobId = queuedExecution.idAsync().join(); @@ -278,13 +282,13 @@ void executesRemotelyUsingNetworkCommunication() { respondWithJobStateResponseWhenJobStateRequestIsSent(jobId, COMPLETED); respondWithJobCancelResponseWhenJobCancelRequestIsSent(jobId, false); - JobExecution execution = computeComponent.executeRemotely( - DEFAULT, remoteNode, List.of(), SimpleJob.class.getName(), null, "a" + JobExecution execution = computeComponent.executeRemotely( + DEFAULT, remoteNode, List.of(), SimpleJob.class.getName(), null, SharedComputeUtils.marshalArgOrResult("a", null) ); - assertThat(execution.resultAsync(), willBe("remoteResponse")); + assertThat(unwrapResult(execution), willBe("remoteResponse")); // Verify that second invocation of resultAsync will not result in the network communication (i.e. the result is cached locally) - assertThat(execution.resultAsync(), willBe("remoteResponse")); + assertThat(unwrapResult(execution), willBe("remoteResponse")); assertThat(execution.stateAsync(), willBe(jobStateWithStatus(COMPLETED))); assertThat(execution.cancelAsync(), willBe(false)); @@ -303,11 +307,11 @@ void getsStateAndCancelsRemotelyUsingNetworkCommunication() { respondWithJobStateResponseWhenJobStateRequestIsSent(jobId, EXECUTING); respondWithJobCancelResponseWhenJobCancelRequestIsSent(jobId, true); - JobExecution execution = computeComponent.executeRemotely(DEFAULT, remoteNode, List.of(), LongJob.class.getName(), null, - null); + JobExecution execution = computeComponent.executeRemotely( + DEFAULT, remoteNode, List.of(), LongJob.class.getName(), null, null); assertThat(execution.stateAsync(), willBe(jobStateWithStatus(EXECUTING))); - assertThat(execution.resultAsync(), willBe("remoteResponse")); + assertThat(unwrapResult(execution), willBe("remoteResponse")); assertThat(execution.cancelAsync(), willBe(true)); assertThatExecuteRequestWasSent(LongJob.class.getName(), null); @@ -322,8 +326,8 @@ void changePriorityRemotelyUsingNetworkCommunication() { respondWithExecuteResponseWhenExecuteRequestIsSent(jobId); respondWithJobChangePriorityResponseWhenJobChangePriorityRequestIsSent(jobId); - JobExecution execution = computeComponent.executeRemotely(DEFAULT, remoteNode, List.of(), LongJob.class.getName(), null, - null); + JobExecution execution = computeComponent.executeRemotely( + DEFAULT, remoteNode, List.of(), LongJob.class.getName(), null, null); assertThat(execution.changePriorityAsync(1), willBe(true)); @@ -340,7 +344,7 @@ private void respondWithExecuteResponseWhenExecuteRequestIsSent(UUID jobId) { private void respondWithJobResultResponseWhenJobResultRequestIsSent(UUID jobId) { JobResultResponse jobResultResponse = new ComputeMessagesFactory().jobResultResponse() - .result("remoteResponse") + .result(SharedComputeUtils.marshalArgOrResult("remoteResponse", null)) .build(); when(messagingService.invoke(anyString(), argThat(msg -> jobResultRequestWithJobId(msg, jobId)), anyLong())) .thenReturn(completedFuture(jobResultResponse)); @@ -406,7 +410,7 @@ private void assertThatExecuteRequestWasSent(String jobClassName, String args) { ExecuteRequest capturedRequest = invokeAndCaptureRequest(ExecuteRequest.class); assertThat(capturedRequest.jobClassName(), is(jobClassName)); - assertThat(capturedRequest.input(), is(equalTo(args))); + assertThat(SharedComputeUtils.unmarshalArgOrResult(capturedRequest.input(), null, null), is(equalTo(args))); } private void assertThatJobResultRequestWasSent(UUID jobId) { @@ -461,7 +465,7 @@ void executesJobAndRespondsWhenGetsExecuteRequest() { .executeOptions(DEFAULT) .deploymentUnits(List.of()) .jobClassName(SimpleJob.class.getName()) - .input("") + .input(SharedComputeUtils.marshalArgOrResult("", null)) .build(); ExecuteResponse executeResponse = sendRequestAndCaptureResponse(executeRequest, testNode, 123L); @@ -474,7 +478,7 @@ void executesJobAndRespondsWhenGetsExecuteRequest() { .build(); JobResultResponse jobResultResponse = sendRequestAndCaptureResponse(jobResultRequest, testNode, 456L); - assertThat(jobResultResponse.result(), is("jobResponse")); + assertThat(SharedComputeUtils.unmarshalArgOrResult(jobResultResponse.result(), null, null), is("jobResponse")); assertThat(jobResultResponse.throwable(), is(nullValue())); } @@ -528,7 +532,7 @@ void stoppedComponentReturnsExceptionOnExecuteRequestAttempt() { .executeOptions(DEFAULT) .deploymentUnits(List.of()) .jobClassName(SimpleJob.class.getName()) - .input(new Object[]{"a", 42}) + .input(SharedComputeUtils.marshalArgOrResult(42, null)) .build(); ExecuteResponse response = sendRequestAndCaptureResponse(request, testNode, 123L); @@ -712,14 +716,20 @@ private CompletableFuture executeLocally(String jobClassName) { } private CompletableFuture executeLocally(List units, String jobClassName, String args) { - return computeComponent.executeLocally(DEFAULT, units, jobClassName, null, args).resultAsync(); + return computeComponent.executeLocally( + DEFAULT, units, jobClassName, null, SharedComputeUtils.marshalArgOrResult(args, null)) + .resultAsync() + .thenApply(r -> SharedComputeUtils.unmarshalArgOrResult(r, null, null)); } private CompletableFuture executeRemotely( String jobClassName, String args ) { - return computeComponent.executeRemotely(DEFAULT, remoteNode, List.of(), jobClassName, null, args).resultAsync(); + return computeComponent.executeRemotely( + DEFAULT, remoteNode, List.of(), jobClassName, null, SharedComputeUtils.marshalArgOrResult(args, null)) + .resultAsync() + .thenApply(r -> SharedComputeUtils.unmarshalArgOrResult(r, null, null)); } private CompletableFuture executeRemotely( @@ -728,6 +738,10 @@ private CompletableFuture executeRemotely( return executeRemotely(jobClassName, null); } + private static CompletableFuture unwrapResult(JobExecution execution) { + return execution.resultAsync().thenApply(r -> SharedComputeUtils.unmarshalArgOrResult(r, null, null)); + } + private static class SimpleJob implements ComputeJob { /** {@inheritDoc} */ @Override diff --git a/modules/compute/src/test/java/org/apache/ignite/internal/compute/IgniteComputeImplTest.java b/modules/compute/src/test/java/org/apache/ignite/internal/compute/IgniteComputeImplTest.java index debd9da2380..88de0e19ee6 100644 --- a/modules/compute/src/test/java/org/apache/ignite/internal/compute/IgniteComputeImplTest.java +++ b/modules/compute/src/test/java/org/apache/ignite/internal/compute/IgniteComputeImplTest.java @@ -119,11 +119,11 @@ void whenNodeIsLocalThenExecutesLocally() { compute.executeAsync( JobTarget.node(localNode), JobDescriptor.builder(JOB_CLASS_NAME).units(testDeploymentUnits).build(), - "a"), + null), willBe("jobResponse") ); - verify(computeComponent).executeLocally(ExecutionOptions.DEFAULT, testDeploymentUnits, JOB_CLASS_NAME, null, "a"); + verify(computeComponent).executeLocally(ExecutionOptions.DEFAULT, testDeploymentUnits, JOB_CLASS_NAME, null, null); } @Test @@ -137,7 +137,7 @@ void safeCallCancelHandleAfterJobProcessing() { JobTarget.node(localNode), JobDescriptor.builder(JOB_CLASS_NAME).units(testDeploymentUnits).build(), cancelHandle.token(), - "a"), + null), willBe("jobResponse") ); @@ -157,11 +157,11 @@ void whenNodeIsLocalAndIdIsChangedThenExecutesLocally() { compute.executeAsync( JobTarget.node(newNode), JobDescriptor.builder(JOB_CLASS_NAME).units(testDeploymentUnits).build(), - "a"), + null), willBe("jobResponse") ); - verify(computeComponent).executeLocally(ExecutionOptions.DEFAULT, testDeploymentUnits, JOB_CLASS_NAME, null, "a"); + verify(computeComponent).executeLocally(ExecutionOptions.DEFAULT, testDeploymentUnits, JOB_CLASS_NAME, null, null); } @Test @@ -189,11 +189,11 @@ void whenNodeIsLocalThenExecutesLocallyWithOptions() { compute.executeAsync( JobTarget.node(localNode), JobDescriptor.builder(JOB_CLASS_NAME).units(testDeploymentUnits).options(options).build(), - "a"), + null), willBe("jobResponse") ); - verify(computeComponent).executeLocally(expectedOptions, testDeploymentUnits, JOB_CLASS_NAME, null, "a"); + verify(computeComponent).executeLocally(expectedOptions, testDeploymentUnits, JOB_CLASS_NAME, null, null); } @Test @@ -250,7 +250,7 @@ void executeBroadcastAsync() { CompletableFuture> future = compute.executeBroadcastAsync( Set.of(localNode, remoteNode), JobDescriptor.builder(JOB_CLASS_NAME).units(testDeploymentUnits).build(), - "a" + null ); assertThat(future, willBe(aMapWithSize(2))); @@ -268,24 +268,24 @@ private void respondWhenAskForPrimaryReplica() { } private void respondWhenExecutingSimpleJobLocally(ExecutionOptions executionOptions) { - when(computeComponent.executeLocally(executionOptions, testDeploymentUnits, JOB_CLASS_NAME, null, "a")) - .thenReturn(completedExecution("jobResponse")); + when(computeComponent.executeLocally(executionOptions, testDeploymentUnits, JOB_CLASS_NAME, null, null)) + .thenReturn(completedExecution(SharedComputeUtils.marshalArgOrResult("jobResponse", null))); } private void respondWhenExecutingSimpleJobLocally(ExecutionOptions executionOptions, CancellationToken token) { - when(computeComponent.executeLocally(executionOptions, testDeploymentUnits, JOB_CLASS_NAME, token, "a")) - .thenReturn(completedExecution("jobResponse")); + when(computeComponent.executeLocally(executionOptions, testDeploymentUnits, JOB_CLASS_NAME, token, null)) + .thenReturn(completedExecution(SharedComputeUtils.marshalArgOrResult("jobResponse", null))); } private void respondWhenExecutingSimpleJobRemotely(ExecutionOptions options) { when(computeComponent.executeRemotelyWithFailover( - eq(remoteNode), any(), eq(testDeploymentUnits), eq(JOB_CLASS_NAME), eq(options), isNull(), eq("a") - )).thenReturn(completedExecution("remoteResponse")); + eq(remoteNode), any(), eq(testDeploymentUnits), eq(JOB_CLASS_NAME), eq(options), isNull(), any() + )).thenReturn(completedExecution(SharedComputeUtils.marshalArgOrResult("remoteResponse", null))); } private void verifyExecuteRemotelyWithFailover(ExecutionOptions options) { verify(computeComponent).executeRemotelyWithFailover( - eq(remoteNode), any(), eq(testDeploymentUnits), eq(JOB_CLASS_NAME), eq(options), isNull(), eq("a") + eq(remoteNode), any(), eq(testDeploymentUnits), eq(JOB_CLASS_NAME), eq(options), isNull(), any() ); } diff --git a/modules/compute/src/test/java/org/apache/ignite/internal/compute/executor/ComputeExecutorTest.java b/modules/compute/src/test/java/org/apache/ignite/internal/compute/executor/ComputeExecutorTest.java index 33c9e46e155..2aac5713266 100644 --- a/modules/compute/src/test/java/org/apache/ignite/internal/compute/executor/ComputeExecutorTest.java +++ b/modules/compute/src/test/java/org/apache/ignite/internal/compute/executor/ComputeExecutorTest.java @@ -44,6 +44,7 @@ import org.apache.ignite.compute.task.MapReduceTask; import org.apache.ignite.compute.task.TaskExecutionContext; import org.apache.ignite.internal.compute.ExecutionOptions; +import org.apache.ignite.internal.compute.SharedComputeUtils; import org.apache.ignite.internal.compute.configuration.ComputeConfiguration; import org.apache.ignite.internal.compute.state.InMemoryComputeStateMachine; import org.apache.ignite.internal.configuration.testframework.ConfigurationExtension; @@ -143,62 +144,60 @@ public CompletableFuture executeAsync(JobExecutionContext context, Obje @Test void retryJobFail() { - AtomicInteger runTimes = new AtomicInteger(); - int maxRetries = 5; + RetryJobFail.runTimes.set(0); JobExecutionInternal execution = computeExecutor.executeJob( ExecutionOptions.builder().maxRetries(maxRetries).build(), RetryJobFail.class, null, - new Object[]{runTimes} + null ); await().until(execution::state, jobStateWithStatus(FAILED)); - assertThat(runTimes.get(), is(maxRetries + 1)); + assertThat(RetryJobFail.runTimes.get(), is(maxRetries + 1)); } - private static class RetryJobFail implements ComputeJob { + private static class RetryJobFail implements ComputeJob { + static final AtomicInteger runTimes = new AtomicInteger(); @Override - public CompletableFuture executeAsync(JobExecutionContext context, Object... args) { - AtomicInteger runTimes = (AtomicInteger) args[0]; + public CompletableFuture executeAsync(JobExecutionContext context, Object args) { runTimes.incrementAndGet(); + throw new RuntimeException(); } } @Test void retryJobSuccess() { - AtomicInteger runTimes = new AtomicInteger(); - int maxRetries = 5; + RetryJobSuccess.runTimes.set(0); JobExecutionInternal execution = computeExecutor.executeJob( ExecutionOptions.builder().maxRetries(maxRetries).build(), RetryJobSuccess.class, null, - new Object[]{ - runTimes, - maxRetries - } + SharedComputeUtils.marshalArgOrResult(maxRetries, null) ); await().until(execution::state, jobStateWithStatus(COMPLETED)); - assertThat(runTimes.get(), is(maxRetries + 1)); + assertThat(RetryJobSuccess.runTimes.get(), is(maxRetries + 1)); } - private static class RetryJobSuccess implements ComputeJob { + private static class RetryJobSuccess implements ComputeJob { + static final AtomicInteger runTimes = new AtomicInteger(); @Override - public CompletableFuture executeAsync(JobExecutionContext context, Object... args) { - AtomicInteger runTimes = (AtomicInteger) args[0]; - int maxRetries = (int) args[1]; + public CompletableFuture executeAsync(JobExecutionContext context, Integer args) { + int maxRetries = args; + if (runTimes.incrementAndGet() <= maxRetries) { throw new RuntimeException(); } + return completedFuture(0); } @@ -206,34 +205,34 @@ public CompletableFuture executeAsync(JobExecutionContext context, Obje @Test void runJobOnce() { - AtomicInteger runTimes = new AtomicInteger(); - int maxRetries = 5; + JobSuccess.runTimes.set(0); JobExecutionInternal execution = computeExecutor.executeJob( ExecutionOptions.builder().maxRetries(maxRetries).build(), JobSuccess.class, null, - runTimes + null ); await().until(execution::state, jobStateWithStatus(COMPLETED)); - assertThat(execution.resultAsync(), willBe(1)); - assertThat(runTimes.get(), is(1)); + assertThat(execution.resultAsync().thenApply(h -> SharedComputeUtils.unmarshalArgOrResult(h, null, null)), willBe(1)); + assertThat(JobSuccess.runTimes.get(), is(1)); } - private static class JobSuccess implements ComputeJob { + private static class JobSuccess implements ComputeJob { + static final AtomicInteger runTimes = new AtomicInteger(); @Override - public CompletableFuture executeAsync(JobExecutionContext context, AtomicInteger runTimes) { + public CompletableFuture executeAsync(JobExecutionContext context, Object arg) { return completedFuture(runTimes.incrementAndGet()); } } @Test void findJobArgumentType() { - assertThat(getJobExecuteArgumentType(JobSuccess.class), is(AtomicInteger.class)); + assertThat(getJobExecuteArgumentType(RetryJobSuccess.class), is(Integer.class)); } private static class Task implements MapReduceTask { diff --git a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/client/ItThinClientComputeMarshallingTest.java b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/client/ItThinClientComputeMarshallingTest.java index c6294801ec0..41836de82b6 100644 --- a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/client/ItThinClientComputeMarshallingTest.java +++ b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/client/ItThinClientComputeMarshallingTest.java @@ -318,7 +318,9 @@ void mapReduce() { + ":marshalledOnClient" + ":unmarshalledOnServer" // Job argument unmarshalled on the target node + ":processedOnServer" // Job processed on the target node - // Job result is not marshalled because it's a local execution + // TODO IGNITE-24183 Avoid job argument and result marshalling on local execution + + ":marshalledOnServer" + + ":unmarshalledOnClient" + ":listMarshalledOnServer" // Reduce job result marshalled on the client handler node + ":listUnmarshalledOnClient", // Reduce job result unmarshalled on the client "Input_1" diff --git a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/client/ItThinClientComputeTypeCheckMarshallingTest.java b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/client/ItThinClientComputeTypeCheckMarshallingTest.java index ed7124fe3b2..2c504d64bc7 100644 --- a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/client/ItThinClientComputeTypeCheckMarshallingTest.java +++ b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/client/ItThinClientComputeTypeCheckMarshallingTest.java @@ -20,6 +20,7 @@ import static java.util.concurrent.CompletableFuture.completedFuture; import static org.apache.ignite.compute.JobStatus.COMPLETED; import static org.apache.ignite.compute.JobStatus.FAILED; +import static org.apache.ignite.internal.testframework.IgniteTestUtils.assertThrowsWithCause; import static org.apache.ignite.internal.testframework.matchers.CompletableFutureExceptionMatcher.willThrow; import static org.apache.ignite.internal.testframework.matchers.CompletableFutureMatcher.willBe; import static org.apache.ignite.internal.testframework.matchers.JobStateMatcher.jobStateWithStatus; @@ -42,7 +43,6 @@ import org.apache.ignite.lang.ErrorGroups.Compute; import org.apache.ignite.marshalling.ByteArrayMarshaller; import org.apache.ignite.marshalling.Marshaller; -import org.apache.ignite.marshalling.UnmarshallingException; import org.apache.ignite.marshalling.UnsupportedObjectTypeMarshallingException; import org.jetbrains.annotations.Nullable; import org.junit.jupiter.api.Test; @@ -77,7 +77,7 @@ void resultMarshallerDefinedOnlyInJob() { ); await().until(result::stateAsync, willBe(jobStateWithStatus(COMPLETED))); - assertThat(result.resultAsync(), willThrow(UnmarshallingException.class)); + assertThat(result.resultAsync(), willThrow(ComputeException.class)); } @Test @@ -111,11 +111,9 @@ void resultMarshallerDoesNotMatch() { ); await().until(result::stateAsync, willBe(jobStateWithStatus(COMPLETED))); + // The job has completed successfully, but result was not unmarshaled - assertThat(result.resultAsync(), willBe(instanceOf(byte[].class))); - assertThrows(ClassCastException.class, () -> { - Integer i = result.resultAsync().join(); - }); + assertThrowsWithCause(() -> result.resultAsync().join(), ClassCastException.class); } static class ArgumentTypeCheckingmarshallingJob implements ComputeJob { diff --git a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/client/ItThinClientPojoComputeMarshallingTest.java b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/client/ItThinClientPojoComputeMarshallingTest.java index 344f9770d2a..3b389e0025e 100644 --- a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/client/ItThinClientPojoComputeMarshallingTest.java +++ b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/client/ItThinClientPojoComputeMarshallingTest.java @@ -24,6 +24,7 @@ import static org.hamcrest.Matchers.is; import org.apache.ignite.client.IgniteClient; +import org.apache.ignite.compute.ComputeException; import org.apache.ignite.compute.JobDescriptor; import org.apache.ignite.compute.JobDescriptor.Builder; import org.apache.ignite.compute.JobTarget; @@ -35,7 +36,6 @@ import org.apache.ignite.internal.runner.app.Jobs.PojoJob; import org.apache.ignite.internal.runner.app.Jobs.PojoResult; import org.apache.ignite.internal.runner.app.Jobs.TwoStringPojo; -import org.apache.ignite.marshalling.UnmarshallingException; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Test; @@ -137,13 +137,13 @@ void pojoJobWithoutResultClass(int targetNodeIdx) { // When run job with custom marshaller for pojo argument and result. assertThrows( - UnmarshallingException.class, + ComputeException.class, () -> client.compute().execute( JobTarget.node(targetNode), JobDescriptor.builder(PojoJob.class).build(), new PojoArg().setIntValue(2).setStrValue("1") ), - "Can not unpack object because the pojo class is not provided but the object was packed as pojo." + "JobDescriptor.resultClass is not defined, but the job result is packed as a POJO" ); } diff --git a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/compute/ItEmbeddedMarshallingTest.java b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/compute/ItEmbeddedMarshallingTest.java index 9d1ece5c46e..090c426a5ab 100644 --- a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/compute/ItEmbeddedMarshallingTest.java +++ b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/compute/ItEmbeddedMarshallingTest.java @@ -103,7 +103,7 @@ void pojoExecOnAnotherNode() { var embeddedCompute = node.compute(); PojoResult result = embeddedCompute.execute( JobTarget.node(targetNode), - JobDescriptor.builder(PojoJob.class).build(), + JobDescriptor.builder(PojoJob.class).resultClass(PojoResult.class).build(), new PojoArg().setIntValue(2).setStrValue("1") ); @@ -122,7 +122,7 @@ void pojoExecOnSame() { var embeddedCompute = node.compute(); PojoResult result = embeddedCompute.execute( JobTarget.node(targetNode), - JobDescriptor.builder(PojoJob.class).build(), + JobDescriptor.builder(PojoJob.class).resultClass(PojoResult.class).build(), new PojoArg().setIntValue(2).setStrValue("1") ); @@ -145,7 +145,8 @@ void local() { "Input" ); - assertEquals("Input:marshalledOnClient:unmarshalledOnServer:processedOnServer", result); + // TODO IGNITE-24183 Avoid job argument and result marshalling on local execution + assertEquals("Input:marshalledOnClient:unmarshalledOnServer:processedOnServer:marshalledOnServer:unmarshalledOnClient", result); } @Test @@ -164,9 +165,10 @@ void broadcastExecute() { ); // Then. + // TODO IGNITE-24183 Avoid job argument and result marshalling on local execution Map resultExpected = Map.of( // todo: "https://issues.apache.org/jira/browse/IGNITE-23024" - node(0), "Input:marshalledOnClient:unmarshalledOnServer:processedOnServer", + node(0), "Input:marshalledOnClient:unmarshalledOnServer:processedOnServer:marshalledOnServer:unmarshalledOnClient", node(1), "Input:marshalledOnClient:unmarshalledOnServer:processedOnServer:marshalledOnServer:unmarshalledOnClient" ); @@ -191,9 +193,10 @@ void broadcastSubmit() { ); // Then. + // TODO IGNITE-24183 Avoid job argument and result marshalling on local execution Map resultExpected = Map.of( // todo: "https://issues.apache.org/jira/browse/IGNITE-23024" - node(0), "Input:marshalledOnClient:unmarshalledOnServer:processedOnServer", + node(0), "Input:marshalledOnClient:unmarshalledOnServer:processedOnServer:marshalledOnServer:unmarshalledOnClient", node(1), "Input:marshalledOnClient:unmarshalledOnServer:processedOnServer:marshalledOnServer:unmarshalledOnClient" ); From f0ef09f5ccde0d975c62fc9e5b003ab3f15811b8 Mon Sep 17 00:00:00 2001 From: Kirill Tkalenko Date: Mon, 13 Jan 2025 14:14:47 +0300 Subject: [PATCH 016/113] IGNITE-24172 IndexMetaStorageRecoveryTest#testMissingDropTableWithUpdateLwmBeforeRestart (#5022) --- .../distributed/index/IndexMetaStorage.java | 49 ++++++++++++++----- .../index/IndexMetaStorageRecoveryTest.java | 2 +- 2 files changed, 39 insertions(+), 12 deletions(-) diff --git a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/index/IndexMetaStorage.java b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/index/IndexMetaStorage.java index c779af32af6..3047cf32b64 100644 --- a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/index/IndexMetaStorage.java +++ b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/index/IndexMetaStorage.java @@ -31,7 +31,9 @@ import static org.apache.ignite.internal.event.EventListener.fromFunction; import static org.apache.ignite.internal.hlc.HybridTimestamp.hybridTimestampToLong; import static org.apache.ignite.internal.lowwatermark.event.LowWatermarkEvent.LOW_WATERMARK_CHANGED; +import static org.apache.ignite.internal.metastorage.dsl.Conditions.and; import static org.apache.ignite.internal.metastorage.dsl.Conditions.exists; +import static org.apache.ignite.internal.metastorage.dsl.Conditions.notExists; import static org.apache.ignite.internal.metastorage.dsl.Conditions.value; import static org.apache.ignite.internal.metastorage.dsl.Operations.noop; import static org.apache.ignite.internal.metastorage.dsl.Operations.put; @@ -80,6 +82,7 @@ import org.apache.ignite.internal.metastorage.Entry; import org.apache.ignite.internal.metastorage.MetaStorageManager; import org.apache.ignite.internal.metastorage.Revisions; +import org.apache.ignite.internal.metastorage.dsl.Operation; import org.apache.ignite.internal.util.Cursor; import org.apache.ignite.internal.versioned.VersionedSerialization; import org.jetbrains.annotations.Nullable; @@ -211,7 +214,7 @@ private CompletableFuture onCatalogIndexCreateEvent(CreateIndexEventPar Catalog catalog = catalog(parameters.catalogVersion()); - return updateAndSaveIndexMetaToMetastore(indexId, indexMeta -> { + return createAndSaveIndexMetaToMetastore(indexId, indexMeta -> { assert indexMeta == null : "indexId=" + indexId + "catalogVersion=" + catalog.version(); return IndexMeta.of(indexId, catalog); @@ -336,7 +339,7 @@ private CompletableFuture recoverIndexMetas() { if (fromMetastore == null) { // We did not have time to save at the index creation event. - futures.add(updateAndSaveIndexMetaToMetastore(indexId, indexMeta -> IndexMeta.of(indexId, catalog))); + futures.add(createAndSaveIndexMetaToMetastore(indexId, indexMeta -> IndexMeta.of(indexId, catalog))); } else if (fromMetastore.catalogVersion() < catalog.version()) { if (!catalogIndexDescriptor.name().equals(fromMetastore.indexName())) { // We did not have time to process the index renaming event. @@ -437,17 +440,22 @@ private static ByteArray indexMetaValueKey(IndexMeta indexMeta) { return ByteArray.fromString(INDEX_META_VALUE_KEY_PREFIX + indexMeta.indexId()); } - private CompletableFuture saveToMetastore(IndexMeta newMeta) { - ByteArray versionKey = indexMetaVersionKey(newMeta); + private CompletableFuture createInMetastore(IndexMeta indexMeta) { + return metaStorageManager.invoke( + notExists(indexMetaVersionKey(indexMeta)), + saveIndexMetaOperations(indexMeta), + List.of(noop()) + ); + } + + private CompletableFuture updateInMetastore(IndexMeta indexMeta) { + ByteArray versionKey = indexMetaVersionKey(indexMeta); // We need to keep order for the comparison to work correctly. - byte[] versionValue = intToBytesKeepingOrder(newMeta.catalogVersion()); + byte[] versionValue = intToBytesKeepingOrder(indexMeta.catalogVersion()); return metaStorageManager.invoke( - value(versionKey).lt(versionValue), - List.of( - put(versionKey, versionValue), - put(indexMetaValueKey(newMeta), VersionedSerialization.toBytes(newMeta, IndexMetaSerializer.INSTANCE)) - ), + and(exists(versionKey), value(versionKey).lt(versionValue)), + saveIndexMetaOperations(indexMeta), List.of(noop()) ); } @@ -462,13 +470,32 @@ private CompletableFuture removeFromMetastore(IndexMeta indexMeta) { ); } + private static List saveIndexMetaOperations(IndexMeta indexMeta) { + // We need to keep order for the comparison to work correctly. + byte[] versionValue = intToBytesKeepingOrder(indexMeta.catalogVersion()); + + return List.of( + put(indexMetaVersionKey(indexMeta), versionValue), + put(indexMetaValueKey(indexMeta), VersionedSerialization.toBytes(indexMeta, IndexMetaSerializer.INSTANCE)) + ); + } + + private CompletableFuture createAndSaveIndexMetaToMetastore( + int indexId, + Function<@Nullable IndexMeta, IndexMeta> updateFunction + ) { + IndexMeta newMeta = indexMetaByIndexId.compute(indexId, (id, indexMeta) -> updateFunction.apply(indexMeta)); + + return createInMetastore(newMeta); + } + private CompletableFuture updateAndSaveIndexMetaToMetastore( int indexId, Function<@Nullable IndexMeta, IndexMeta> updateFunction ) { IndexMeta newMeta = indexMetaByIndexId.compute(indexId, (id, indexMeta) -> updateFunction.apply(indexMeta)); - return saveToMetastore(newMeta); + return updateInMetastore(newMeta); } private int lwmCatalogVersion(@Nullable HybridTimestamp lwm) { diff --git a/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/index/IndexMetaStorageRecoveryTest.java b/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/index/IndexMetaStorageRecoveryTest.java index a6c94f6eccb..9665df801af 100644 --- a/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/index/IndexMetaStorageRecoveryTest.java +++ b/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/index/IndexMetaStorageRecoveryTest.java @@ -88,7 +88,7 @@ MetaStorageManager createMetastore() { scheduledExecutorService ); - return StandaloneMetaStorageManager.create(keyValueStorage, readOperationForCompactionTracker); + return StandaloneMetaStorageManager.create(keyValueStorage, clock, readOperationForCompactionTracker); } @Override From 86fd823a47524f46369fd957375fe959cfa93643 Mon Sep 17 00:00:00 2001 From: Pavel Tupitsyn Date: Mon, 13 Jan 2025 15:38:09 +0200 Subject: [PATCH 017/113] IGNITE-24162 .NET: Fix ComputeTests on Windows (#5033) * Windows-specific test failures fixed * CI project added: https://ci.ignite.apache.org/buildConfiguration/ApacheIgnite3xGradle_Test_RunNetWindowsTests/8771669 --- .../Compute/ComputeTests.cs | 32 ++++++++++++------- 1 file changed, 21 insertions(+), 11 deletions(-) diff --git a/modules/platforms/dotnet/Apache.Ignite.Tests/Compute/ComputeTests.cs b/modules/platforms/dotnet/Apache.Ignite.Tests/Compute/ComputeTests.cs index 530a5eb8745..44586787ba4 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Tests/Compute/ComputeTests.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Tests/Compute/ComputeTests.cs @@ -599,7 +599,7 @@ public async Task TestDelayedJobExecutionThrowsWhenConnectionFails() public async Task TestJobExecutionStatusExecuting() { const int sleepMs = 3000; - var beforeStart = SystemClock.Instance.GetCurrentInstant(); + var beforeStart = GetCurrentInstant(); var jobExecution = await Client.Compute.SubmitAsync(await GetNodeAsync(1), SleepJob, sleepMs); @@ -610,7 +610,7 @@ public async Task TestJobExecutionStatusExecuting() public async Task TestJobExecutionStatusCompleted() { const int sleepMs = 1; - var beforeStart = SystemClock.Instance.GetCurrentInstant(); + var beforeStart = GetCurrentInstant(); var jobExecution = await Client.Compute.SubmitAsync(await GetNodeAsync(1), SleepJob, sleepMs); await jobExecution.GetResultAsync(); @@ -621,7 +621,7 @@ public async Task TestJobExecutionStatusCompleted() [Test] public async Task TestJobExecutionStatusFailed() { - var beforeStart = SystemClock.Instance.GetCurrentInstant(); + var beforeStart = GetCurrentInstant(); var jobExecution = await Client.Compute.SubmitAsync(await GetNodeAsync(1), ErrorJob, "unused"); Assert.CatchAsync(async () => await jobExecution.GetResultAsync()); @@ -644,7 +644,7 @@ public async Task TestJobExecutionStatusNull() public async Task TestJobExecutionCancel() { const int sleepMs = 5000; - var beforeStart = SystemClock.Instance.GetCurrentInstant(); + var beforeStart = GetCurrentInstant(); var jobExecution = await Client.Compute.SubmitAsync(await GetNodeAsync(1), SleepJob, sleepMs); await jobExecution.CancelAsync(); @@ -722,7 +722,7 @@ public async Task TestBigDecimalPropagation(string number, int scale) [Test] public async Task TestMapReduceNodeNameTask() { - Instant beforeStart = SystemClock.Instance.GetCurrentInstant(); + Instant beforeStart = GetCurrentInstant(); ITaskExecution taskExec = await Client.Compute.SubmitMapReduceAsync(NodeNameTask, "+arg"); @@ -747,9 +747,9 @@ public async Task TestMapReduceNodeNameTask() Assert.IsNotNull(state); Assert.AreEqual(TaskStatus.Completed, state.Status); Assert.AreEqual(taskExec.Id, state.Id); - Assert.That(state.CreateTime, Is.GreaterThan(beforeStart)); - Assert.That(state.StartTime, Is.GreaterThan(state.CreateTime)); - Assert.That(state.FinishTime, Is.GreaterThan(state.StartTime)); + Assert.That(state.CreateTime, Is.GreaterThanOrEqualTo(beforeStart)); + Assert.That(state.StartTime, Is.GreaterThanOrEqualTo(state.CreateTime)); + Assert.That(state.FinishTime, Is.GreaterThanOrEqualTo(state.StartTime)); // Job states. IList jobStates = await taskExec.GetJobStatesAsync(); @@ -894,12 +894,12 @@ private static async Task AssertJobStatus(IJobExecution jobExecution, JobS Assert.IsNotNull(state); Assert.AreEqual(jobExecution.Id, state!.Id); Assert.AreEqual(status, state.Status); - Assert.Greater(state.CreateTime, beforeStart); - Assert.Greater(state.StartTime, state.CreateTime); + Assert.That(state.CreateTime, Is.GreaterThanOrEqualTo(beforeStart)); + Assert.That(state.StartTime, Is.GreaterThanOrEqualTo(state.CreateTime)); if (status is JobStatus.Canceled or JobStatus.Completed or JobStatus.Failed) { - Assert.Greater(state.FinishTime, state.StartTime); + Assert.That(state.FinishTime, Is.GreaterThanOrEqualTo(state.StartTime)); } else { @@ -907,6 +907,16 @@ private static async Task AssertJobStatus(IJobExecution jobExecution, JobS } } + private static Instant GetCurrentInstant() + { + var instant = SystemClock.Instance.GetCurrentInstant(); + + // Subtract 1 milli to account for OS-specific time resolution differences in .NET and Java. + return OperatingSystem.IsWindows() + ? instant.Minus(Duration.FromMilliseconds(1)) + : instant; + } + private async Task> GetNodeAsync(int index) => JobTarget.Node( (await Client.GetClusterNodesAsync()).OrderBy(n => n.Name).Skip(index).First()); From 4202f60962a4470f463531ba325ced3f20b28cad Mon Sep 17 00:00:00 2001 From: Cyrill Date: Mon, 13 Jan 2025 17:27:22 +0300 Subject: [PATCH 018/113] IGNITE-23885 Start partition in a common manner in case of a single element in chain (#5012) --- .../rebalance/RebalanceUtil.java | 67 ++++++++++++++++-- .../table/distributed/TableManager.java | 68 ++++++++++++++----- 2 files changed, 113 insertions(+), 22 deletions(-) diff --git a/modules/distribution-zones/src/main/java/org/apache/ignite/internal/distributionzones/rebalance/RebalanceUtil.java b/modules/distribution-zones/src/main/java/org/apache/ignite/internal/distributionzones/rebalance/RebalanceUtil.java index c845b177636..1f1ec672ac9 100644 --- a/modules/distribution-zones/src/main/java/org/apache/ignite/internal/distributionzones/rebalance/RebalanceUtil.java +++ b/modules/distribution-zones/src/main/java/org/apache/ignite/internal/distributionzones/rebalance/RebalanceUtil.java @@ -63,6 +63,7 @@ import org.apache.ignite.internal.metastorage.dsl.Iif; import org.apache.ignite.internal.partitiondistribution.Assignment; import org.apache.ignite.internal.partitiondistribution.Assignments; +import org.apache.ignite.internal.partitiondistribution.AssignmentsChain; import org.apache.ignite.internal.replicator.TablePartitionId; import org.apache.ignite.internal.util.ExceptionUtils; import org.jetbrains.annotations.Nullable; @@ -620,6 +621,24 @@ public static CompletableFuture> partitionAssignments( .thenApply(e -> (e.value() == null) ? null : Assignments.fromBytes(e.value()).nodes()); } + /** + * Returns partition assignments from meta storage locally. + * + * @param metaStorageManager Meta storage manager. + * @param tablePartitionId Table partition id. + * @param revision Revision. + * @return Returns partition assignments from meta storage locally or {@code null} if assignments is absent. + */ + public static @Nullable Assignments stableAssignmentsGetLocally( + MetaStorageManager metaStorageManager, + TablePartitionId tablePartitionId, + long revision + ) { + Entry entry = metaStorageManager.getLocally(stablePartAssignmentsKey(tablePartitionId), revision); + + return (entry == null || entry.empty() || entry.tombstone()) ? null : Assignments.fromBytes(entry.value()); + } + /** * Returns partition assignments from meta storage locally. * @@ -636,9 +655,9 @@ public static Set partitionAssignmentsGetLocally( int partitionNumber, long revision ) { - Entry entry = metaStorageManager.getLocally(stablePartAssignmentsKey(new TablePartitionId(tableId, partitionNumber)), revision); + Assignments assignments = stableAssignmentsGetLocally(metaStorageManager, new TablePartitionId(tableId, partitionNumber), revision); - return (entry == null || entry.empty() || entry.tombstone()) ? null : Assignments.fromBytes(entry.value()).nodes(); + return assignments == null ? null : assignments.nodes(); } /** @@ -686,11 +705,11 @@ public static List tableAssignmentsGetLocally( ) { return IntStream.range(0, numberOfPartitions) .mapToObj(p -> { - Entry e = metaStorageManager.getLocally(stablePartAssignmentsKey(new TablePartitionId(tableId, p)), revision); + Assignments assignments = stableAssignmentsGetLocally(metaStorageManager, new TablePartitionId(tableId, p), revision); - assert e != null && !e.empty() && !e.tombstone() : e; + assert assignments != null; - return Assignments.fromBytes(e.value()); + return assignments; }) .collect(toList()); } @@ -718,4 +737,42 @@ public static List tablePendingAssignmentsGetLocally( }) .collect(toList()); } + + /** + * Returns assignments chains for all table partitions from meta storage locally. + * + * @param metaStorageManager Meta storage manager. + * @param tableId Table id. + * @param numberOfPartitions Number of partitions. + * @param revision Revision. + * @return Future with table assignments as a value. + */ + public static List tableAssignmentsChainGetLocally( + MetaStorageManager metaStorageManager, + int tableId, + int numberOfPartitions, + long revision + ) { + return IntStream.range(0, numberOfPartitions) + .mapToObj(p -> assignmentsChainGetLocally(metaStorageManager, new TablePartitionId(tableId, p), revision)) + .collect(toList()); + } + + /** + * Returns assignments chain from meta storage locally. + * + * @param metaStorageManager Meta storage manager. + * @param tablePartitionId Table partition id. + * @param revision Revision. + * @return Returns assignments chain from meta storage locally or {@code null} if assignments is absent. + */ + public static @Nullable AssignmentsChain assignmentsChainGetLocally( + MetaStorageManager metaStorageManager, + TablePartitionId tablePartitionId, + long revision + ) { + Entry e = metaStorageManager.getLocally(assignmentsChainKey(tablePartitionId), revision); + + return e != null ? AssignmentsChain.fromBytes(e.value()) : null; + } } diff --git a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/TableManager.java b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/TableManager.java index e2a9f94fdbf..dc5a8883b43 100644 --- a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/TableManager.java +++ b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/TableManager.java @@ -33,12 +33,15 @@ import static org.apache.ignite.internal.distributionzones.rebalance.RebalanceUtil.ASSIGNMENTS_SWITCH_REDUCE_PREFIX_BYTES; import static org.apache.ignite.internal.distributionzones.rebalance.RebalanceUtil.PENDING_ASSIGNMENTS_PREFIX_BYTES; import static org.apache.ignite.internal.distributionzones.rebalance.RebalanceUtil.STABLE_ASSIGNMENTS_PREFIX_BYTES; +import static org.apache.ignite.internal.distributionzones.rebalance.RebalanceUtil.assignmentsChainGetLocally; import static org.apache.ignite.internal.distributionzones.rebalance.RebalanceUtil.assignmentsChainKey; import static org.apache.ignite.internal.distributionzones.rebalance.RebalanceUtil.extractTablePartitionId; import static org.apache.ignite.internal.distributionzones.rebalance.RebalanceUtil.partitionAssignmentsGetLocally; import static org.apache.ignite.internal.distributionzones.rebalance.RebalanceUtil.pendingPartAssignmentsKey; +import static org.apache.ignite.internal.distributionzones.rebalance.RebalanceUtil.stableAssignmentsGetLocally; import static org.apache.ignite.internal.distributionzones.rebalance.RebalanceUtil.stablePartAssignmentsKey; import static org.apache.ignite.internal.distributionzones.rebalance.RebalanceUtil.subtract; +import static org.apache.ignite.internal.distributionzones.rebalance.RebalanceUtil.tableAssignmentsChainGetLocally; import static org.apache.ignite.internal.distributionzones.rebalance.RebalanceUtil.tableAssignmentsGetLocally; import static org.apache.ignite.internal.distributionzones.rebalance.RebalanceUtil.tablePendingAssignmentsGetLocally; import static org.apache.ignite.internal.distributionzones.rebalance.RebalanceUtil.union; @@ -304,7 +307,7 @@ public class TableManager implements IgniteTablesInternal, IgniteComponent { * Versioned value for tracking RAFT groups initialization and starting completion. * *

Only explicitly updated in - * {@link #startLocalPartitionsAndClients(CompletableFuture, List, TableImpl, boolean, long)}. + * {@link #startLocalPartitionsAndClients(CompletableFuture, List, List, TableImpl, boolean, long)}. * *

Completed strictly after {@link #localPartitionsVv}. */ @@ -1132,6 +1135,7 @@ private CompletableFuture onTableRename(RenameTableEventParameters parameters private CompletableFuture startLocalPartitionsAndClients( CompletableFuture> stableAssignmentsFuture, List<@Nullable Assignments> pendingAssignmentsForPartitions, + List<@Nullable AssignmentsChain> assignmentsChains, TableImpl table, boolean isRecovery, long assignmentsTimestamp @@ -1161,13 +1165,23 @@ private CompletableFuture startLocalPartitionsAndClients( boolean shouldStartPartition; if (isRecovery) { - // The condition to start the replica is - // `pending.contains(node) || (stable.contains(node) && !pending.isForce())`. - // However we check only the right part of this condition here - // since after `startTables` we have a call to `processAssignmentsOnRecovery`, - // which executes pending assignments update and will start required partitions there. - shouldStartPartition = localMemberAssignmentInStable != null - && (pendingAssignments == null || !pendingAssignments.force()); + AssignmentsChain assignmentsChain = assignmentsChains.get(i); + + if (lastRebalanceWasGraceful(assignmentsChain)) { + // The condition to start the replica is + // `pending.contains(node) || (stable.contains(node) && !pending.isForce())`. + // However we check only the right part of this condition here + // since after `startTables` we have a call to `processAssignmentsOnRecovery`, + // which executes pending assignments update and will start required partitions there. + shouldStartPartition = localMemberAssignmentInStable != null + && (pendingAssignments == null || !pendingAssignments.force()); + } else { + // TODO: Use logic from https://issues.apache.org/jira/browse/IGNITE-23874 + LOG.warn("Recovery after a forced rebalance for table is not supported yet [tableId={}, partitionId={}].", + tableId, partId); + shouldStartPartition = localMemberAssignmentInStable != null + && (pendingAssignments == null || !pendingAssignments.force()); + } } else { shouldStartPartition = localMemberAssignmentInStable != null; } @@ -1663,6 +1677,9 @@ private CompletableFuture createTableLocally( List pendingAssignments = tablePendingAssignmentsGetLocally(metaStorageMgr, tableId, zoneDescriptor.partitions(), causalityToken); + List assignmentsChains = + tableAssignmentsChainGetLocally(metaStorageMgr, tableId, zoneDescriptor.partitions(), causalityToken); + CompletableFuture> stableAssignmentsFutureAfterInvoke = writeTableAssignmentsToMetastore(tableId, zoneDescriptor.consistencyMode(), stableAssignmentsFuture); @@ -1674,6 +1691,7 @@ private CompletableFuture createTableLocally( zoneDescriptor, stableAssignmentsFutureAfterInvoke, pendingAssignments, + assignmentsChains, onNodeRecovery, catalog.time() ); @@ -1696,6 +1714,7 @@ private CompletableFuture createTableLocally( CatalogZoneDescriptor zoneDescriptor, CompletableFuture> stableAssignmentsFuture, List pendingAssignments, + List assignmentsChains, boolean onNodeRecovery, long assignmentsTimestamp ) { @@ -1748,6 +1767,7 @@ private CompletableFuture createTableLocally( return startLocalPartitionsAndClients( stableAssignmentsFuture, pendingAssignments, + assignmentsChains, table, onNodeRecovery, assignmentsTimestamp @@ -2152,7 +2172,9 @@ private CompletableFuture handleChangePendingAssignmentEvent( TablePartitionId replicaGrpId = extractTablePartitionId(pendingAssignmentsEntry.key(), PENDING_ASSIGNMENTS_PREFIX_BYTES); // Stable assignments from the meta store, which revision is bounded by the current pending event. - Assignments stableAssignments = stableAssignments(replicaGrpId, revision); + Assignments stableAssignments = stableAssignmentsGetLocally(metaStorageMgr, replicaGrpId, revision); + + AssignmentsChain assignmentsChain = assignmentsChainGetLocally(metaStorageMgr, replicaGrpId, revision); Assignments pendingAssignments = Assignments.fromBytes(pendingAssignmentsEntry.value()); @@ -2195,6 +2217,7 @@ private CompletableFuture handleChangePendingAssignmentEvent( table, stableAssignments, pendingAssignments, + assignmentsChain, revision, isRecovery ).thenAccept(v -> executeIfLocalNodeIsPrimaryForGroup( @@ -2218,6 +2241,7 @@ private CompletableFuture handleChangePendingAssignmentEvent( TableImpl tbl, @Nullable Assignments stableAssignments, Assignments pendingAssignments, + @Nullable AssignmentsChain assignmentsChain, long revision, boolean isRecovery ) { @@ -2235,7 +2259,14 @@ private CompletableFuture handleChangePendingAssignmentEvent( // `pending.contains(node) || (stable.contains(node) && !pending.isForce())`. // This condition covers the left part of the OR expression. // The right part of it is covered in `startLocalPartitionsAndClients`. - shouldStartLocalGroupNode = localMemberAssignmentInPending != null; + if (lastRebalanceWasGraceful(assignmentsChain)) { + shouldStartLocalGroupNode = localMemberAssignmentInPending != null; + } else { + // TODO: Use logic from https://issues.apache.org/jira/browse/IGNITE-23874. + LOG.warn("Recovery after a forced rebalance for table is not supported yet [tablePartitionId={}].", + replicaGrpId); + shouldStartLocalGroupNode = localMemberAssignmentInPending != null; + } } else { shouldStartLocalGroupNode = localMemberAssignmentInPending != null && localMemberAssignmentInStable == null; } @@ -2353,6 +2384,15 @@ private CompletableFuture handleChangePendingAssignmentEvent( }), ioExecutor); } + /** + * For HA zones: Check that last rebalance was graceful (caused by common rebalance triggers, like data nodes change, replica factor + * change, etc.) rather than forced (caused by a disaster recovery reset after losing the majority of nodes). + */ + private static boolean lastRebalanceWasGraceful(@Nullable AssignmentsChain assignmentsChain) { + // Assignments chain is either empty (when there have been no stable switch yet) or contains a single element in chain. + return assignmentsChain == null || assignmentsChain.chain().size() == 1; + } + private static PartitionSet extendPartitionSet(@Nullable PartitionSet oldPartitionSet, int partitionId) { PartitionSet newPartitionSet = Objects.requireNonNullElseGet(oldPartitionSet, BitSetPartitionSet::new); newPartitionSet.set(partitionId); @@ -2980,7 +3020,7 @@ public CompletableFuture restartPartition(TablePartitionId tablePartitionI TableImpl table = tables.get(tablePartitionId.tableId()); return stopPartitionForRestart(tablePartitionId, table).thenComposeAsync(unused1 -> { - Assignments stableAssignments = stableAssignments(tablePartitionId, revision); + Assignments stableAssignments = stableAssignmentsGetLocally(metaStorageMgr, tablePartitionId, revision); assert stableAssignments != null : "tablePartitionId=" + tablePartitionId + ", revision=" + revision; @@ -3005,12 +3045,6 @@ public CompletableFuture restartPartition(TablePartitionId tablePartitionI }), ioExecutor)); } - private @Nullable Assignments stableAssignments(TablePartitionId tablePartitionId, long revision) { - Entry entry = metaStorageMgr.getLocally(stablePartAssignmentsKey(tablePartitionId), revision); - - return Assignments.fromBytes(entry.value()); - } - @Override public void setStreamerReceiverRunner(StreamerReceiverRunner runner) { this.streamerReceiverRunner = runner; From 2e3aaf5c34ca4164eabbdf52b524aedd1e49a50d Mon Sep 17 00:00:00 2001 From: Max Zhuravkov Date: Mon, 13 Jan 2025 17:59:34 +0200 Subject: [PATCH 019/113] IGNITE-23947: Sql. Add a test to check operator signatures (#5014) --- .../sql/docs/DocumentedOperators.java | 393 ++++++++++ .../sql/docs/DocumentedOperatorsSelfTest.java | 106 +++ .../internal/sql/docs/OperatorListTest.java | 435 +++++++++++ .../src/test/resources/docs/operator_list.txt | 714 ++++++++++++++++++ 4 files changed, 1648 insertions(+) create mode 100644 modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/docs/DocumentedOperators.java create mode 100644 modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/docs/DocumentedOperatorsSelfTest.java create mode 100644 modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/docs/OperatorListTest.java create mode 100644 modules/sql-engine/src/test/resources/docs/operator_list.txt diff --git a/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/docs/DocumentedOperators.java b/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/docs/DocumentedOperators.java new file mode 100644 index 00000000000..5ec35dcc629 --- /dev/null +++ b/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/docs/DocumentedOperators.java @@ -0,0 +1,393 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.sql.docs; + +import static org.junit.jupiter.api.Assertions.fail; + +import java.io.PrintWriter; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.Set; +import java.util.regex.Matcher; +import java.util.regex.Pattern; +import java.util.stream.Collectors; +import org.apache.calcite.runtime.SqlFunctions; +import org.apache.calcite.sql.SqlOperandCountRange; +import org.apache.calcite.sql.SqlOperator; +import org.apache.calcite.sql.SqlOperatorTable; +import org.apache.calcite.sql.SqlSyntax; + +/** Collection of operators to validate SQL operator signatures. */ +class DocumentedOperators { + + private static final Pattern TYPE_NAME_PATTERN = Pattern.compile("<[^>]*>"); + + private final String name; + + private final List operators = new ArrayList<>(); + + private boolean include = true; + + /** Creates a collection of operators. */ + DocumentedOperators(String name) { + this.name = name; + } + + /** Adds a public operator. */ + DocumentedOperator add(SqlOperator operator) { + return addOp(operator, operator.getName(), false); + } + + /** Adds a public operator under the specified name. */ + DocumentedOperator add(SqlOperator operator, String syntaxName) { + return addOp(operator, syntaxName, false); + } + + /** Adds an internal operator. */ + DocumentedOperator internal(SqlOperator operator) { + return addOp(operator, operator.getName(), true); + } + + /** Adds an internal operator under the specified name. */ + DocumentedOperator internal(SqlOperator operator, String syntaxName) { + return addOp(operator, syntaxName, true); + } + + /** Marks this collection of operators as excluded from a comparison. */ + DocumentedOperators exclude() { + include = false; + return this; + } + + private DocumentedOperator addOp(SqlOperator operator, String syntaxName, boolean internal) { + DocumentedOperator op = new DocumentedOperator(operator, syntaxName, internal); + operators.add(op); + return op; + } + + static void printOperators(PrintWriter pw, List operators) { + operators.stream().filter(ops -> ops.include).forEach(ops -> { + pw.println("=== " + ops.name); + pw.println(); + ops.operators.forEach(op -> printSignature(pw, op)); + pw.println(); + }); + pw.flush(); + } + + /** Validates the given operator table against a list of operators. */ + static void validateOperatorList( + SqlOperatorTable operatorTable, + List operators, + String resourceName + ) { + Set annotated = operators.stream() + .flatMap(ops -> ops.operators.stream().map(f -> f.operator)).collect(Collectors.toSet()); + + List notDocumented = new ArrayList<>(); + + for (SqlOperator operator : operatorTable.getOperatorList()) { + if (!annotated.contains(operator)) { + notDocumented.add(operator); + } + } + + if (!notDocumented.isEmpty()) { + StringBuilder sb = new StringBuilder(); + + sb.append("Signatures of the following operators are missing from ") + .append(resourceName) + .append(". Add these operators to as public (add(OperatorTable.MY_OP))") + .append(" or include them as internal (call .hide(OperatorTable.MY_OP):") + .append(System.lineSeparator()) + .append("Review the difference and file an issue to update SQL documentation if necessary.") + .append(System.lineSeparator()); + + notDocumented.forEach(o -> sb.append(describeOperator(o)).append(System.lineSeparator())); + + fail(sb.toString()); + } + } + + private static String describeOperator(SqlOperator operator) { + return operator.getName() + " class: " + operator.getClass().getCanonicalName(); + } + + /** Operator. */ + static class DocumentedOperator { + final SqlOperator operator; + + final String syntaxName; + + final boolean internal; + + final List sigs = new ArrayList<>(); + + DocumentedOperator(SqlOperator operator, String displayName, boolean internal) { + this.operator = operator; + this.syntaxName = displayName; + this.internal = internal; + } + + /** Adds a signature. */ + DocumentedOperator sig(String sigStr) { + Signature sig = new Signature(syntaxName, sigStr); + sigs.add(sig); + return this; + } + } + + static void printSignature(PrintWriter pw, DocumentedOperator op) { + Signatures sigs = Signatures.makeSignatures(op); + + if (op.internal) { + pw.println("[internal]"); + } + + for (String sig : sigs.sigs) { + pw.print(sig); + if (sigs.manual) { + pw.print(" ***"); + } + pw.println(); + } + pw.println("#" + sigs.hash); + pw.println(); + } + + static class Signatures { + + final List sigs; + + final String hash; + + final boolean manual; + + Signatures(List sigs, String hash, boolean manual) { + this.sigs = sigs; + this.hash = hash; + this.manual = manual; + } + + static Signatures makeSignatures(DocumentedOperator op) { + String opHash = computeOperatorHash(op); + + if (!op.sigs.isEmpty()) { + List sigs = op.sigs.stream() + .map(s -> s.fullSig) + .collect(Collectors.toList()); + + return new Signatures(sigs, opHash, true); + } + + try { + String allowedSignatures = op.operator.getAllowedSignatures(); + String[] signatures; + + String opName = op.operator.getName(); + + // Coverts "' op '" into + // Specifically handles > / < + if ("<".equals(opName) || ">".equals(opName)) { + signatures = allowedSignatures.replace("'", "") + .replace(" " + opName + " ", " ") + .split("\\n"); + } else { + signatures = allowedSignatures.replace("'", "") + .replace(opName, "") + .split("\\n"); + } + + List signaturesList = SignatureFormat.formatAll(SignatureFormat.ARGS, op, signatures); + List sigs; + + if (signaturesList.isEmpty()) { + sigs = List.of(op.syntaxName); + } else if (signaturesList.size() == 1) { + sigs = List.of(SignatureFormat.FULL.format(op, signatures[0])); + } else { + sigs = new ArrayList<>(SignatureFormat.formatAll(SignatureFormat.FULL, op, signatures)); + } + + return new Signatures(sigs, opHash, false); + } catch (Throwable error) { + String syntaxName = op.syntaxName + " "; + + return new Signatures(List.of(syntaxName), opHash, false); + } + } + } + + private static class Signature { + + final String fullSig; + + final String argSig; + + Signature(String opName, String sigStr) { + Matcher matcher = TYPE_NAME_PATTERN.matcher(sigStr); + List params = new ArrayList<>(); + + while (matcher.find()) { + String type = matcher.group(); + params.add(type); + } + + this.fullSig = opName + sigStr; + this.argSig = params.stream().collect(Collectors.joining(", ", "(", ")")); + } + } + + enum SignatureFormat { + /** Operator name ands arguments. Example: for {@code f(a, b)} full format produces {@code f(a, b)}. */ + FULL, + /** Only arguments. Example: for {@code f(a, b}} this format produces {@code (a, b)}. */ + ARGS; + + static List formatAll(SignatureFormat format, DocumentedOperator op, String[] sigs) { + return Arrays.stream(sigs) + .map(DocumentedOperators::formatTypeNames) + .map(s -> format.format(op, s)) + .distinct() + .collect(Collectors.toList()); + } + + String format(DocumentedOperator op, String sig) { + switch (this) { + case FULL: + return full(op, sig); + case ARGS: + return args(op, sig); + default: + throw new IllegalArgumentException("Unknown format: " + this); + } + } + + private static String full(DocumentedOperator op, String in) { + String s = formatTypeNames(in); + SqlSyntax syntax = op.operator.getSyntax(); + + if (syntax == SqlSyntax.PREFIX) { + return op.syntaxName + " " + s; + } else if (syntax == SqlSyntax.POSTFIX) { + return s + op.syntaxName; + } else if (syntax == SqlSyntax.BINARY) { + String[] args = s.split("\\s+"); + return args[0] + " " + op.syntaxName + " " + args[1]; + } else if (syntax == SqlSyntax.FUNCTION_ID) { + return op.syntaxName; + } else { + return op.syntaxName + s; + } + } + + private static String args(DocumentedOperator op, String in) { + String s = formatTypeNames(in); + SqlSyntax syntax = op.operator.getSyntax(); + + if (syntax == SqlSyntax.BINARY) { + String[] args = s.split("\\s+"); + return args[0] + " " + args[1]; + } else if (syntax == SqlSyntax.FUNCTION_ID) { + return ""; + } else { + return s; + } + } + } + + /** Converts {@code FUNC ( AND )} into {@code FUNC ( AND )}}. */ + static String formatTypeNames(String input) { + Matcher matcher = TYPE_NAME_PATTERN.matcher(input); + + StringBuilder result = new StringBuilder(); + + while (matcher.find()) { + String matchedSubstring = matcher.group(); + String replacement = matchedSubstring.toLowerCase(); + + matcher.appendReplacement(result, replacement); + } + + matcher.appendTail(result); + + return result.toString(); + } + + /** Computes hash of SQL operator. */ + private static String computeOperatorHash(DocumentedOperator op) { + SqlOperator sqlOp = op.operator; + + StringBuilder sb = new StringBuilder(); + + sb.append("name=").append(op.syntaxName) + .append("opName=").append(sqlOp.getName()) + .append("syntax=").append(sqlOp.getSyntax()) + .append("kind=").append(sqlOp.kind) + .append("signatures="); + + try { + String allowedSigs = sqlOp.getAllowedSignatures(); + sb.append(allowedSigs); + } catch (Throwable t) { + // Assume that an operator signature changes, if an error message changes. + sb.append("'); + } + + // We can not use neither operandTypeChecker nor operandTypeInference because those properties + // can be provided as lambdas that does not have stable textual representation. + + sb.append("operandCountRange="); + // Assume that changes to operandCountRange are reflected in its properties. + try { + SqlOperandCountRange range = sqlOp.getOperandCountRange(); + sb.append(range.getMin()).append('-').append(range.getMax()); + } catch (UnsupportedOperationException ignore) { + // Change from no operandCountRange to some operandCountRange should mean signatures might have been modified. + sb.append("not_implemented"); + } + + // Include negated operator + SqlOperator negated = sqlOp.not(); + if (negated != null) { + sb.append("negated=").append(negated.getName()); + } + + // Include reverse operator + SqlOperator reverse = sqlOp.reverse(); + if (reverse != null) { + sb.append("reverse=").append(reverse.getName()); + } + + // Other properties + + sb.append("leftPrecedence=").append(sqlOp.getLeftPrec()) + .append("rightPrecedence=").append(sqlOp.getRightPrec()) + .append("symmetrical=").append(sqlOp.isSymmetrical()) + .append("deterministic=").append(sqlOp.isDeterministic()) + .append("aggregator=").append(sqlOp.isAggregator()) + .append("group=").append(sqlOp.isGroup()) + .append("groupAuxiliary=").append(sqlOp.isGroupAuxiliary()) + .append("allowsFraming=").append(sqlOp.allowsFraming()) + .append("requiresOrder=").append(sqlOp.requiresOrder()) + .append("requiresOver=").append(sqlOp.requiresOver()); + + return SqlFunctions.sha1(sb.toString()); + } +} diff --git a/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/docs/DocumentedOperatorsSelfTest.java b/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/docs/DocumentedOperatorsSelfTest.java new file mode 100644 index 00000000000..47f95e74e04 --- /dev/null +++ b/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/docs/DocumentedOperatorsSelfTest.java @@ -0,0 +1,106 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.sql.docs; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.containsString; +import static org.hamcrest.Matchers.not; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.fail; + +import java.util.List; +import org.apache.calcite.sql.SqlOperator; +import org.apache.calcite.sql.SqlOperatorTable; +import org.apache.calcite.sql.fun.SqlStdOperatorTable; +import org.apache.calcite.sql.util.SqlOperatorTables; +import org.apache.ignite.internal.sql.docs.DocumentedOperators.DocumentedOperator; +import org.apache.ignite.internal.sql.docs.DocumentedOperators.Signatures; +import org.apache.ignite.internal.testframework.BaseIgniteAbstractTest; +import org.hamcrest.Matchers; +import org.junit.jupiter.api.Test; +import org.opentest4j.AssertionFailedError; + +/** Tests for {@link DocumentedOperators}. */ +public class DocumentedOperatorsSelfTest extends BaseIgniteAbstractTest { + + @Test + public void testFormatting() { + assertEquals(" > ", DocumentedOperators.formatTypeNames(" > ")); + assertEquals("!", DocumentedOperators.formatTypeNames("!")); + assertEquals("- ", DocumentedOperators.formatTypeNames("- ")); + assertEquals("Func()", DocumentedOperators.formatTypeNames("Func()")); + assertEquals("Func(, )", DocumentedOperators.formatTypeNames("Func(, )")); + + assertEquals( + List.of(" < "), + Signatures.makeSignatures(new DocumentedOperator(SqlStdOperatorTable.LESS_THAN, "<", true)).sigs + ); + assertEquals( + List.of(" > "), + Signatures.makeSignatures(new DocumentedOperator(SqlStdOperatorTable.GREATER_THAN, ">", true)).sigs + ); + assertEquals( + List.of("MIN()"), + Signatures.makeSignatures(new DocumentedOperator(SqlStdOperatorTable.MIN, "MIN", true)).sigs + ); + } + + @Test + public void testValidationFunctionOk() { + SqlOperator add = SqlStdOperatorTable.PLUS; + SqlOperator union = SqlStdOperatorTable.UNION; + SqlOperator sub = SqlStdOperatorTable.MINUS; + + SqlOperatorTable table = SqlOperatorTables.of(add, sub, union); + + DocumentedOperators ops = new DocumentedOperators("Test"); + ops.add(add); + ops.internal(union); + ops.add(sub); + + DocumentedOperators.validateOperatorList(table, List.of(ops), "some file"); + } + + @Test + public void testValidationFunctionFailsWhenSomeOpsAreNotIncluded() { + SqlOperator add = SqlStdOperatorTable.PLUS; + SqlOperator mul = SqlStdOperatorTable.MULTIPLY; + SqlOperator sub = SqlStdOperatorTable.MINUS; + + SqlOperatorTable table = SqlOperatorTables.of(add, sub, mul); + + DocumentedOperators ops = new DocumentedOperators("Test"); + ops.add(add); + ops.add(mul); + + // validateOperatorList should fail because minus operator is missing from DocumentedOperators. + try { + DocumentedOperators.validateOperatorList(table, List.of(ops), "some file"); + } catch (AssertionFailedError err) { + + assertThat("Error:\n" + err.getMessage(), err.getMessage(), Matchers.allOf( + containsString("- class: " + sub.getClass().getCanonicalName()), + not(containsString("+ class: " + add.getClass().getCanonicalName())), + not(containsString("* class: " + mul.getClass().getCanonicalName())) + )); + return; + } + + fail(); + } +} diff --git a/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/docs/OperatorListTest.java b/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/docs/OperatorListTest.java new file mode 100644 index 00000000000..f87aa735bf4 --- /dev/null +++ b/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/docs/OperatorListTest.java @@ -0,0 +1,435 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.sql.docs; + +import static org.junit.jupiter.api.Assertions.assertEquals; + +import java.io.IOException; +import java.io.PrintWriter; +import java.io.StringWriter; +import java.nio.charset.StandardCharsets; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.util.List; +import org.apache.calcite.sql.fun.SqlInternalOperators; +import org.apache.calcite.sql.fun.SqlLibraryOperators; +import org.apache.calcite.sql.fun.SqlStdOperatorTable; +import org.apache.ignite.internal.sql.engine.sql.fun.IgniteSqlOperatorTable; +import org.apache.ignite.internal.testframework.BaseIgniteAbstractTest; +import org.junit.jupiter.api.Test; + +/** + * Checks that all operators defined in a table of SQL operators have matching signatures. + * + *

When sql operator table changes, this test should detect the following errors: + *

    + *
  • An operator was added
  • + *
  • An operator was removed
  • + *
  • An operator signature was changed. This is achieved by storing a SHA-1 hash of operator signatures./li> + *
+ * + *

The list of documented operators is stored in {@link OperatorListTest#OPERATOR_LIST} in the following format: + *

+ *   F1(<numeric>)
+ *   F1(<string>)
+ *   #81a63101fa89dfb9b9115265fb519d7dda810aa2
+ *
+ *   F2(<string>) ***
+ *   F2() ***
+ *   #7f27a84679525580ba4fa549407c28de073c0f69
+ * 
+ * + *

In this file a list of signatures for each operator is followed by a SHA-1 hash. + * When a signature ends with {@code ***} it means that it was not automatically generated and was added manually. + * Each internal SQL operator (an operator that can not be created from public API) as a {@code [Internal]} + * on a line prior to their definition. + */ +public class OperatorListTest extends BaseIgniteAbstractTest { + + private static final String OPERATOR_LIST = "src/test/resources/docs/operator_list.txt"; + + private final IgniteSqlOperatorTable operatorTable = new IgniteSqlOperatorTable(); + + private final List allOperators = List.of( + stringFunctions(), + numericFunctions(), + dateTimeFunctions(), + aggregateFunctions(), + otherFunctions(), + regexFunctions(), + jsonFunctions(), + xmlFunctions(), + structAndCollectionFunctions(), + + // These operators added but should not included in the docs + setOperators(), + logicalOperators() + ); + + @Test + public void test() throws IOException { + DocumentedOperators.validateOperatorList(operatorTable, allOperators, OPERATOR_LIST); + + StringWriter sw = new StringWriter(); + + try (PrintWriter pw = new PrintWriter(sw)) { + DocumentedOperators.printOperators(pw, allOperators); + + Path path = Paths.get(OPERATOR_LIST); + List lines = Files.readAllLines(path, StandardCharsets.UTF_8); + + assertEquals( + String.join("\n", lines).stripTrailing(), + sw.toString().stripTrailing(), + "operator list does not match" + ); + } + } + + // Operator definitions + + private static DocumentedOperators stringFunctions() { + DocumentedOperators ops = new DocumentedOperators("String Functions"); + + ops.add(SqlStdOperatorTable.UPPER); + ops.add(SqlStdOperatorTable.LOWER); + ops.add(SqlStdOperatorTable.INITCAP); + ops.add(SqlLibraryOperators.TO_BASE64); + ops.add(SqlLibraryOperators.FROM_BASE64); + ops.add(SqlLibraryOperators.MD5); + ops.add(SqlLibraryOperators.SHA1); + ops.add(SqlStdOperatorTable.SUBSTRING); + ops.add(SqlLibraryOperators.LEFT); + ops.add(SqlLibraryOperators.RIGHT); + ops.add(SqlStdOperatorTable.REPLACE); + ops.add(SqlLibraryOperators.TRANSLATE3); + ops.add(SqlLibraryOperators.CHR); + ops.add(SqlStdOperatorTable.CHAR_LENGTH); + ops.add(SqlStdOperatorTable.CHARACTER_LENGTH); + ops.add(SqlStdOperatorTable.CONCAT).sig("(, , ..."); + ops.add(SqlLibraryOperators.CONCAT_FUNCTION); + ops.add(SqlStdOperatorTable.OVERLAY); + ops.add(SqlStdOperatorTable.POSITION); + ops.add(SqlStdOperatorTable.ASCII); + ops.add(SqlLibraryOperators.REPEAT); + ops.add(SqlLibraryOperators.SPACE); + ops.add(SqlLibraryOperators.STRCMP); + ops.add(SqlLibraryOperators.SOUNDEX); + ops.add(SqlLibraryOperators.DIFFERENCE); + ops.add(SqlLibraryOperators.REVERSE); + ops.add(SqlStdOperatorTable.TRIM); + ops.add(SqlLibraryOperators.LTRIM); + ops.add(SqlLibraryOperators.RTRIM); + ops.add(IgniteSqlOperatorTable.SUBSTR); + ops.add(IgniteSqlOperatorTable.LENGTH); + ops.add(IgniteSqlOperatorTable.OCTET_LENGTH); + + // LIKE and SIMILAR. + ops.add(SqlStdOperatorTable.LIKE); + ops.internal(SqlStdOperatorTable.NOT_LIKE); + ops.add(SqlStdOperatorTable.SIMILAR_TO); + ops.internal(SqlStdOperatorTable.NOT_SIMILAR_TO); + + return ops; + } + + private static DocumentedOperators regexFunctions() { + DocumentedOperators ops = new DocumentedOperators("Regular Expression Functions"); + + // POSIX REGEX. + ops.add(SqlStdOperatorTable.POSIX_REGEX_CASE_INSENSITIVE, "~*"); + ops.add(SqlStdOperatorTable.POSIX_REGEX_CASE_SENSITIVE, "~"); + ops.internal(SqlStdOperatorTable.NEGATED_POSIX_REGEX_CASE_INSENSITIVE); + ops.internal(SqlStdOperatorTable.NEGATED_POSIX_REGEX_CASE_SENSITIVE); + + ops.add(SqlLibraryOperators.REGEXP_REPLACE_2); + ops.add(SqlLibraryOperators.REGEXP_REPLACE_3); + ops.add(SqlLibraryOperators.REGEXP_REPLACE_4); + ops.add(SqlLibraryOperators.REGEXP_REPLACE_5); + ops.add(SqlLibraryOperators.REGEXP_REPLACE_6); + + return ops; + } + + private static DocumentedOperators numericFunctions() { + DocumentedOperators ops = new DocumentedOperators("Numeric Functions"); + + // Math functions. + ops.add(SqlStdOperatorTable.MOD); // Arithmetic remainder. + ops.add(SqlStdOperatorTable.EXP); // Euler's number e raised to the power of a value. + ops.add(SqlStdOperatorTable.POWER); + ops.add(SqlStdOperatorTable.LN); // Natural logarithm. + ops.add(SqlStdOperatorTable.LOG10); // The base 10 logarithm. + ops.add(SqlStdOperatorTable.ABS); // Absolute value. + ops.add(SqlStdOperatorTable.RAND); // Random. + ops.add(SqlStdOperatorTable.RAND_INTEGER); // Integer random. + ops.add(SqlStdOperatorTable.ACOS); // Arc cosine. + ops.add(SqlStdOperatorTable.ASIN); // Arc sine. + ops.add(SqlStdOperatorTable.ATAN); // Arc tangent. + ops.add(SqlStdOperatorTable.ATAN2); // Angle from coordinates. + ops.add(SqlStdOperatorTable.SQRT); // Square root. + ops.add(SqlStdOperatorTable.CBRT); // Cube root. + ops.add(SqlStdOperatorTable.COS); // Cosine + ops.add(SqlLibraryOperators.COSH); // Hyperbolic cosine. + ops.add(SqlStdOperatorTable.COT); // Cotangent. + ops.add(SqlStdOperatorTable.DEGREES); // Radians to degrees. + ops.add(SqlStdOperatorTable.RADIANS); // Degrees to radians. + ops.add(IgniteSqlOperatorTable.ROUND); // Fixes return type scale. + ops.add(SqlStdOperatorTable.SIGN); + ops.add(SqlStdOperatorTable.SIN); // Sine. + ops.add(SqlLibraryOperators.SINH); // Hyperbolic sine. + ops.add(SqlStdOperatorTable.TAN); // Tangent. + ops.add(SqlLibraryOperators.TANH); // Hyperbolic tangent. + ops.add(IgniteSqlOperatorTable.TRUNCATE); // Fixes return type scale. + ops.add(SqlStdOperatorTable.PI); + + return ops; + } + + private static DocumentedOperators dateTimeFunctions() { + DocumentedOperators ops = new DocumentedOperators("Date/time Functions"); + + // Date and time. + ops.internal(SqlStdOperatorTable.DATETIME_PLUS); + ops.internal(SqlStdOperatorTable.MINUS_DATE); + ops.add(SqlStdOperatorTable.EXTRACT); + ops.add(SqlStdOperatorTable.FLOOR); + ops.add(SqlStdOperatorTable.CEIL); + ops.add(SqlStdOperatorTable.TIMESTAMP_ADD); + ops.add(SqlStdOperatorTable.TIMESTAMP_DIFF); + ops.add(SqlStdOperatorTable.LAST_DAY); + ops.add(SqlLibraryOperators.DAYNAME); + ops.add(SqlLibraryOperators.MONTHNAME); + ops.add(SqlStdOperatorTable.DAYOFMONTH); + ops.add(SqlStdOperatorTable.DAYOFWEEK); + ops.add(SqlStdOperatorTable.DAYOFYEAR); + ops.add(SqlStdOperatorTable.YEAR); + ops.add(SqlStdOperatorTable.QUARTER); + ops.add(SqlStdOperatorTable.MONTH); + ops.add(SqlStdOperatorTable.WEEK); + ops.add(SqlStdOperatorTable.HOUR); + ops.add(SqlStdOperatorTable.MINUTE); + ops.add(SqlStdOperatorTable.SECOND); + ops.add(SqlLibraryOperators.TIMESTAMP_SECONDS); // Seconds since 1970-01-01 to timestamp. + ops.add(SqlLibraryOperators.TIMESTAMP_MILLIS); // Milliseconds since 1970-01-01 to timestamp. + ops.add(SqlLibraryOperators.TIMESTAMP_MICROS); // Microseconds since 1970-01-01 to timestamp. + ops.add(SqlLibraryOperators.UNIX_SECONDS); // Timestamp to seconds since 1970-01-01. + ops.add(SqlLibraryOperators.UNIX_MILLIS); // Timestamp to milliseconds since 1970-01-01. + ops.add(SqlLibraryOperators.UNIX_MICROS); // Timestamp to microseconds since 1970-01-01. + ops.add(SqlLibraryOperators.UNIX_DATE); // Date to days since 1970-01-01. + ops.add(SqlLibraryOperators.DATE_FROM_UNIX_DATE); // Days since 1970-01-01 to date. + ops.add(SqlLibraryOperators.DATE) + .sig("()") + .sig("(, , )") + .sig("()") + .sig("()") + .sig("(, )"); + + // Current time functions. + ops.add(SqlStdOperatorTable.CURRENT_TIME); + ops.add(SqlStdOperatorTable.CURRENT_TIMESTAMP); + ops.add(SqlStdOperatorTable.CURRENT_DATE); + ops.add(SqlStdOperatorTable.LOCALTIME); + ops.add(SqlStdOperatorTable.LOCALTIMESTAMP); + + return ops; + } + + private static DocumentedOperators aggregateFunctions() { + DocumentedOperators ops = new DocumentedOperators("Aggregate Functions"); + + ops.add(SqlStdOperatorTable.COUNT) + .sig("(*)") + .sig("()"); + + ops.add(SqlStdOperatorTable.SUM); + ops.add(SqlStdOperatorTable.SUM0, "SUM0"); + ops.add(SqlStdOperatorTable.AVG); + // internal function + ops.internal(IgniteSqlOperatorTable.DECIMAL_DIVIDE); + ops.add(SqlStdOperatorTable.MIN); + ops.add(SqlStdOperatorTable.MAX); + ops.add(SqlStdOperatorTable.ANY_VALUE); + ops.add(SqlStdOperatorTable.SINGLE_VALUE); + ops.internal(SqlStdOperatorTable.FILTER); + + ops.add(SqlStdOperatorTable.EVERY); + ops.add(SqlStdOperatorTable.SOME); + + ops.internal(SqlInternalOperators.LITERAL_AGG); + + return ops; + } + + private static DocumentedOperators jsonFunctions() { + DocumentedOperators ops = new DocumentedOperators("JSON Functions"); + + // JSON Operators + ops.add(SqlStdOperatorTable.JSON_TYPE_OPERATOR); + ops.add(SqlStdOperatorTable.JSON_VALUE_EXPRESSION); + ops.add(SqlStdOperatorTable.JSON_VALUE); + ops.add(SqlStdOperatorTable.JSON_QUERY); + ops.add(SqlLibraryOperators.JSON_TYPE); + ops.add(SqlStdOperatorTable.JSON_EXISTS); + ops.add(SqlLibraryOperators.JSON_DEPTH); + ops.add(SqlLibraryOperators.JSON_KEYS); + ops.add(SqlLibraryOperators.JSON_PRETTY); + ops.add(SqlLibraryOperators.JSON_LENGTH); + ops.add(SqlLibraryOperators.JSON_REMOVE); + ops.add(SqlLibraryOperators.JSON_STORAGE_SIZE); + ops.add(SqlStdOperatorTable.JSON_OBJECT) + .sig("( : , ...)") + .sig("(KEY , VALUE , ...)"); + ops.add(SqlStdOperatorTable.JSON_ARRAY); + ops.add(SqlStdOperatorTable.IS_JSON_VALUE); + ops.add(SqlStdOperatorTable.IS_JSON_OBJECT); + ops.add(SqlStdOperatorTable.IS_JSON_ARRAY); + ops.add(SqlStdOperatorTable.IS_JSON_SCALAR); + ops.internal(SqlStdOperatorTable.IS_NOT_JSON_VALUE); + ops.internal(SqlStdOperatorTable.IS_NOT_JSON_OBJECT); + ops.internal(SqlStdOperatorTable.IS_NOT_JSON_ARRAY); + ops.internal(SqlStdOperatorTable.IS_NOT_JSON_SCALAR); + + return ops; + } + + private static DocumentedOperators otherFunctions() { + DocumentedOperators ops = new DocumentedOperators("Other Functions"); + + ops.add(IgniteSqlOperatorTable.GREATEST2, "GREATEST"); + ops.add(IgniteSqlOperatorTable.LEAST2, "LEAST"); + + // Other functions and operators. + ops.add(SqlStdOperatorTable.CAST).sig("( AS )"); + ops.add(SqlLibraryOperators.INFIX_CAST, "::").sig("::"); + ops.add(SqlStdOperatorTable.COALESCE); + ops.add(SqlLibraryOperators.NVL); + ops.add(SqlStdOperatorTable.NULLIF); + ops.add(SqlStdOperatorTable.CASE).sig("(...)"); + ops.add(SqlLibraryOperators.DECODE); + ops.add(SqlLibraryOperators.LEAST); + ops.add(SqlLibraryOperators.GREATEST); + ops.add(SqlLibraryOperators.COMPRESS); + ops.internal(SqlStdOperatorTable.DEFAULT); + ops.internal(SqlStdOperatorTable.REINTERPRET).sig("()").sig("()"); + + // Exists. + ops.internal(SqlStdOperatorTable.EXISTS); + + // NULLS ordering. + ops.internal(SqlStdOperatorTable.NULLS_FIRST); + ops.internal(SqlStdOperatorTable.NULLS_LAST); + ops.internal(SqlStdOperatorTable.DESC); + + // Ignite + + ops.add(IgniteSqlOperatorTable.TYPEOF); + ops.add(IgniteSqlOperatorTable.RAND_UUID); + ops.add(IgniteSqlOperatorTable.SYSTEM_RANGE); + + return ops; + } + + private static DocumentedOperators structAndCollectionFunctions() { + DocumentedOperators ops = new DocumentedOperators("Collection Functions"); + + // ROW constructor + ops.add(SqlStdOperatorTable.ROW); + + // Collections. + ops.add(SqlStdOperatorTable.MAP_VALUE_CONSTRUCTOR); + ops.add(SqlStdOperatorTable.ARRAY_VALUE_CONSTRUCTOR); + + ops.add(SqlStdOperatorTable.ITEM, "") + .sig("array[]") + .sig("map[]") + .sig("struct[ | ]"); + + ops.add(SqlStdOperatorTable.CARDINALITY); + ops.add(SqlStdOperatorTable.IS_EMPTY); + ops.internal(SqlStdOperatorTable.IS_NOT_EMPTY); + + return ops; + } + + private static DocumentedOperators xmlFunctions() { + DocumentedOperators ops = new DocumentedOperators("XML Functions"); + + // XML Operators. + ops.add(SqlLibraryOperators.EXTRACT_VALUE); + ops.add(SqlLibraryOperators.XML_TRANSFORM); + ops.add(SqlLibraryOperators.EXTRACT_XML); + ops.add(SqlLibraryOperators.EXISTS_NODE); + + return ops; + } + + private static DocumentedOperators setOperators() { + DocumentedOperators ops = new DocumentedOperators("Set Operators").exclude(); + + // Set operators. + ops.add(SqlStdOperatorTable.UNION); + ops.add(SqlStdOperatorTable.UNION_ALL); + ops.add(SqlStdOperatorTable.EXCEPT); + ops.add(SqlStdOperatorTable.EXCEPT_ALL); + ops.add(SqlStdOperatorTable.INTERSECT); + ops.add(SqlStdOperatorTable.INTERSECT_ALL); + + return ops; + } + + private static DocumentedOperators logicalOperators() { + DocumentedOperators ops = new DocumentedOperators("Logical Operators"); + + // Logical. + ops.add(SqlStdOperatorTable.AND); + ops.add(SqlStdOperatorTable.OR); + ops.add(SqlStdOperatorTable.NOT); + // Comparisons. + ops.add(SqlStdOperatorTable.LESS_THAN); + ops.add(SqlStdOperatorTable.LESS_THAN_OR_EQUAL); + ops.add(SqlStdOperatorTable.GREATER_THAN); + ops.add(SqlStdOperatorTable.GREATER_THAN_OR_EQUAL); + ops.add(SqlStdOperatorTable.EQUALS); + ops.add(SqlStdOperatorTable.NOT_EQUALS); + ops.add(SqlStdOperatorTable.BETWEEN, "").sig("BETWEEN AND "); + ops.internal(SqlStdOperatorTable.NOT_BETWEEN, "").sig("NOT BETWEEN AND "); + // Arithmetic. + ops.add(SqlStdOperatorTable.PLUS); + ops.add(SqlStdOperatorTable.MINUS); + ops.add(SqlStdOperatorTable.MULTIPLY); + ops.add(SqlStdOperatorTable.DIVIDE); + ops.internal(SqlStdOperatorTable.DIVIDE_INTEGER); // Used internally. + ops.add(SqlStdOperatorTable.PERCENT_REMAINDER); + ops.add(SqlStdOperatorTable.UNARY_MINUS); + ops.add(SqlStdOperatorTable.UNARY_PLUS); + + // IS ... operator. + ops.add(SqlStdOperatorTable.IS_NULL); + ops.add(SqlStdOperatorTable.IS_NOT_NULL); + ops.add(SqlStdOperatorTable.IS_TRUE); + ops.add(SqlStdOperatorTable.IS_NOT_TRUE); + ops.add(SqlStdOperatorTable.IS_FALSE); + ops.add(SqlStdOperatorTable.IS_NOT_FALSE); + ops.add(SqlStdOperatorTable.IS_DISTINCT_FROM); + ops.add(SqlStdOperatorTable.IS_NOT_DISTINCT_FROM); + + return ops; + } +} diff --git a/modules/sql-engine/src/test/resources/docs/operator_list.txt b/modules/sql-engine/src/test/resources/docs/operator_list.txt new file mode 100644 index 00000000000..01ea6ae4556 --- /dev/null +++ b/modules/sql-engine/src/test/resources/docs/operator_list.txt @@ -0,0 +1,714 @@ +=== String Functions + +UPPER() +#9965cbef3eac2aa06c2c308cde3306a10cc4708c + +LOWER() +#f53246b5467a2990e7ae390080a97688780ac01d + +INITCAP() +#3923c4e36530879fb3cb7c86063f457d1206bda4 + +TO_BASE64() +TO_BASE64() +#9b8ad332597f15f16996ed51265c4b8f9c27dd2c + +FROM_BASE64() +#7cedbde3812de934f20cc03fc1727aa5e58c4580 + +MD5() +MD5() +#c30aa309bd67a407e50ad42789d6bd9196ca55f3 + +SHA1() +SHA1() +#8c9019321337e6a8db7f534fbfc5e3a089a30dca + +SUBSTRING( FROM ) +SUBSTRING( FROM FOR ) +SUBSTRING( FROM ) +SUBSTRING( FROM FOR ) +SUBSTRING( FROM ) +SUBSTRING( FROM FOR ) +SUBSTRING( FROM ) +SUBSTRING( FROM FOR ) +#6a239800b4612d21b464be0486d211bf3e33062b + +LEFT(, ) +LEFT(, ) +#3c0e9d41a3688c458661620123e4af0da58ef000 + +RIGHT(, ) +RIGHT(, ) +#1b06f6cd3e91f637e4dafd8f4f888b12809c4094 + +REPLACE(, , ) +#efdb024f3144bf05085158a32dc36c2162a62dc5 + +TRANSLATE3(, , ) +#7d95d4a63a97fbcf8d4c5ba189f0b5dda8b42c93 + +CHR() +#e58f004886b184a9c5276851764e73b1d6f2685b + +CHAR_LENGTH() +#7652cd84f32db99fecafc7af33ffec2c646ee875 + +CHARACTER_LENGTH() +#10185e6642eb58334815d26b83df4f323ff3192a + +||(, , ... *** +#bdec9fef3f8c1927e224d1aae0aba623e33f5a5e + +CONCAT() +#d3879de10e0ef0b0b2b164bd8b925c48e03c357e + +OVERLAY( PLACING FROM ) +OVERLAY( PLACING FROM FOR ) +#cab69b41e0fd0f1dee69df00f6a68f99b5136ba6 + +POSITION( IN ) +POSITION( IN FROM ) +POSITIONINSTR(, , , ) +#cc624511ec052b74c240aa3d9b9c55f4ccbb73d2 + +ASCII() +#ac06d05870d3d429551195b9de8d481c91f96b6e + +REPEAT(, ) +#3e9a907402f5a3a7fe13d20b3e431e3053935534 + +SPACE() +#46cb7470e44e52910513de4ab13592464f17fab2 + +STRCMP(, ) +#1d6f7f5e49c09ba35dd7c52ede8df46e998fb195 + +SOUNDEX() +#9404006d8c9c5e0f77943d1d534a847f5682dc9a + +DIFFERENCE(, ) +#c669e44fdba8ed77b603d05f732a4e495ba9d5fa + +REVERSE() +#b930f4d31bd636153cb00f617ab85a1af6bb276c + +TRIM([BOTH|LEADING|TRAILING] FROM ) +#617d06e21a00df81bf46ab0a8105fc29566eabc8 + +LTRIM() +#1fc9911f5d56b6a977aeab6ab3db3afb77acc0f0 + +RTRIM() +#6f6da6f4ee4c4c144fa4e100e9b9be063531f469 + +SUBSTR(, ) +SUBSTR(, , ) +#354f9745fc48ec99e84addb6dcd0b210e4c70722 + +LENGTH() +LENGTH() +#1817179bf02ba7181678a35a59580513e2a7635f + +OCTET_LENGTH() +OCTET_LENGTH() +#f01708d42209f8026c0590c2bf4e662aedf627e4 + +LIKE(, , ) +#f9618e61dc6e1f47563b2f34babb93222e268195 + +[internal] +NOT LIKE(, , ) +#98a3a61de0c758cd62599fae6293068ac9f8fbf1 + +SIMILAR TO(, , ) +#5b59aa4a2ad6c123016ec3e78d11d440caedf1d8 + +[internal] +NOT SIMILAR TO(, , ) +#4ea1dd2d99dd0ab4b4238969282571665897bb32 + + +=== Numeric Functions + +MOD(, ) +#aba0003fb00b9e43eb24424eaf5c85a854b2a2e1 + +EXP() +#8a79a26006764a33dc590b6c628b0aa142d0a2ae + +POWER(, ) +#7e0a6fe4d6ff7b7e36c3e854a90acd627374e272 + +LN() +#63c5cbcdacff64b235d3c01ed71f7bb6d2b046b6 + +LOG10() +#e4eddf1e750f842761b497d23a0bf9d03eb91736 + +ABS() +ABS() +#9e502d4b4609fbb699a01aabebea2e8e43f9c4cb + +RAND() +RAND() +#73686a8255e539d8b8b8f92832fcaa27cea61bcc + +RAND_INTEGER() +RAND_INTEGER(, ) +#4775cc357178fea27b6becdc05df2d736c2bba23 + +ACOS() +#680b1cb63da746f9ae73522e179db7b9c38c3d9c + +ASIN() +#eabcfde87516431fb77ea93c0e2a301407899d80 + +ATAN() +#c77e02d06b466927530c490b93263442f536b503 + +ATAN2(, ) +#7a385648cbd00a0bf8156a13337bb5d397dc6b4b + +SQRT() +#b2d9db431676b6c062fc2c7a244f8fa622a38c26 + +CBRT() +#426c8824727d6c104c77fbfa440ccfa676fc72c5 + +COS() +#a4c6094e6f572e90158215e02785cc38ec80f67f + +COSH() +#76816c0a69f02f4a36d1b36083df5690d4bdf94d + +COT() +#904aa5293684f22c492753b56c29499d8cbd51e7 + +DEGREES() +#68b0a8cb054ce1384fde8b6999d478f36622b764 + +RADIANS() +#2f99d48ab779bcf15dad0161ba1fa92d24fa0162 + +ROUND() +ROUND(, ) +#a92bed3987231f81459de0de4aed66b6f04dc032 + +SIGN() +#0c41b5e91f5ed43fdbd41f263ab64d90a9e974c9 + +SIN() +#0e40a684c403456fe2f98b57d7449afd738818a8 + +SINH() +#f34057b3faf6514968980e8884a893fba47d7efd + +TAN() +#2301fb49f3e5b9888baa5b5b808b52f44e0fb121 + +TANH() +#e9366c724144704ac20c7596c8754d0733d81cdd + +TRUNCATE() +TRUNCATE(, ) +#87e27c45e666c2cb0b783ee2fa03509d1eee54aa + +PI +#920dd92d06ed99b0327a5bd9bb0ae95b206d656d + + +=== Date/time Functions + +[internal] ++(, , ) +#5dcfb49a7f0aeb5c97045bfe841015075c50108f + +[internal] +-(, , ) +#df54f7f0a3dd9bee9173ef4839b2a564d08dc798 + +EXTRACT( FROM ) +EXTRACT( FROM ) +EXTRACT( FROM ) +#63e623f48be1be04786946166bc839c344ff911d + +FLOOR() +FLOOR() +FLOOR( TO ) +FLOOR(