Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

IGNITE-17859 Update indexes on data modifications #1191

Merged
merged 31 commits into from
Oct 27, 2022

Conversation

korlov42
Copy link
Contributor

No description provided.


assert indexRow != null;

locks[idx++] = lockManager.acquire(txId, new LockKey(indexId, indexRow), LockMode.X);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Not that simple.Different type of indexes will require different type of locks on different operations on different entries

different operations
Insertion in sorted non-unique index expect following locks to be taken:

  • IX_short(nextKey)
  • X_commit(currentKey) if nextKey previously locked in S, X or SIX mode
  • IX_commit(currentKey) otherwise

Select
S_commit(currentKey)

different entries
More precisely select in non-unique is not always about locking current key, but sometimes is about locking +INF

  • If the currentKey is bigger than the upper bound, return NOT_FOUND but take a lock anyway.
  • If currentKey matches the upper bound no need to take a lock on the next key.
  • If the upper bound is null, take a lock on +INF.
  • If nothing is found, take a lock on +INF.

Different type of indexes
That part is pretty straight forward, hash index on insert requires different set of locks to be taken in comparison to sorted index, same is about unique/non-unique indexes.

In order to properly hide given difficulties, I'd favor using Alexey's proposal with wrapping IndexStorages with decorators that'll add index specific lock stuff to put, get and remove methods. That's why, I would expect
Supplier<List<IndexStorage>> activeIndexes to return list of such decorator instead of IndexStorage's.

- rename local variable in RocksDbTableStorage#getOrCreateHashIndex
- move schema descriptor to a constant in ItInternalTableScanTest
- add missing param description to javadoc
- remove unused code
- fix failed tests
@korlov42 korlov42 marked this pull request as ready for review October 24, 2022 07:52
@@ -380,6 +403,50 @@ private SortedIndexDescriptor convert(SortedIndexView indexView) {
);
}

private CompletableFuture<BinaryTuple> toIndexKey(TableIndexView indexView, BinaryRow tableRow) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This one is very inefficient. We're planning to redo it in the future, right?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Decided to rework this a bit. Now converter should be created once for each schema

* @param indexId An index id os the index to register.
* @param searchRowResolver Function which converts given table row to an index key.
*/
public void registerHashIndex(UUID tableId, UUID indexId, Function<BinaryRow, CompletableFuture<BinaryTuple>> searchRowResolver) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Why does it have to be so complicated? Is this because of our weird modules dependencies and the fact that indexes are in their own module?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

yep, it is

BinaryTuplePrefix prefix = BinaryTuplePrefix.fromBinaryTuple(key);

// find next key
Cursor<IndexRow> cursor = storage.scan(prefix, null, SortedIndexStorage.GREATER);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Note for the future: we should add a hint that only one entry will be read. This will allow us to make some optimizations in the storage

* @param <T> A type of the value returned by action.
* @return A future object representing the result of the given action.
*/
private <T> CompletableFuture<T> resolveRowByPk(
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This method has the same name and comment as the previous one. But it does some locks magic. Should we rename it and update the documentation?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Well, I assume (mistakenly) this should be obvious from functions signature: resolve by key and timestamp => RO => just read the resolve the row; resolve by key and txId => RW => acquire all necessary lock before resolving

- rename variable in test storage implementation
- remove artifacts of merge conflict
- fix artifacts of copy-paste in IndexLocker interface
# Conflicts:
#	modules/table/src/main/java/org/apache/ignite/internal/table/distributed/replicator/PartitionReplicaListener.java
#	modules/table/src/main/java/org/apache/ignite/internal/table/distributed/storage/InternalTableImpl.java
#	modules/table/src/test/java/org/apache/ignite/internal/table/distributed/raft/PartitionCommandListenerTest.java
#	modules/table/src/test/java/org/apache/ignite/internal/table/impl/DummyInternalTableImpl.java
# Conflicts:
#	modules/table/src/integrationTest/java/org/apache/ignite/distributed/ItAbstractInternalTableScanTest.java
#	modules/table/src/integrationTest/java/org/apache/ignite/distributed/ItTablePersistenceTest.java
#	modules/table/src/main/java/org/apache/ignite/internal/table/distributed/TableManager.java
#	modules/table/src/main/java/org/apache/ignite/internal/table/distributed/raft/PartitionListener.java
#	modules/table/src/main/java/org/apache/ignite/internal/table/distributed/replicator/PartitionReplicaListener.java
#	modules/table/src/test/java/org/apache/ignite/internal/table/distributed/replication/PartitionReplicaListenerTest.java
* @param <T> Type of the value.
*/
public class Lazy<T> {
private static final Supplier<?> EMPTY = () -> {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Why do we need empty supplier?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

To replace the one provided in construct. The supplier may prevent the object be collected by garbage collector

throw new IllegalStateException("Should not be called");
};

private volatile Supplier<T> supplier;
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Seems that final should be enough.

Copy link
Contributor Author

@korlov42 korlov42 Oct 26, 2022

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

not enough, if want to help GC to collect captured objects

@korlov42 korlov42 changed the title WIP: add row to indexes on insert IGNITE-17859 Update indexes on data modifications Oct 26, 2022
@korlov42 korlov42 merged commit 6f6c3f4 into apache:main Oct 27, 2022
@korlov42 korlov42 deleted the ignite-17859 branch October 27, 2022 15:09
Flaugh24 pushed a commit to unisonteam/ignite-3 that referenced this pull request Nov 4, 2022
lowka pushed a commit to gridgain/apache-ignite-3 that referenced this pull request Mar 18, 2023
lowka pushed a commit to gridgain/apache-ignite-3 that referenced this pull request Apr 19, 2023
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
4 participants