This is an automated email from the ASF dual-hosted git repository.
tkalkirill pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/ignite-3.git
The following commit(s) were added to refs/heads/main by this push:
new cd7a275484 IGNITE-19547 Switch index IDs from UUID to int (#2112)
cd7a275484 is described below
commit cd7a275484f8078b9b0dfd5bb31f411c187a0dcc
Author: Kirill Tkalenko <[email protected]>
AuthorDate: Mon May 29 16:20:49 2023 +0300
IGNITE-19547 Switch index IDs from UUID to int (#2112)
---
.../ignite/client/fakes/FakeInternalTable.java | 39 +++++++-----------
.../apache/ignite/internal/index/HashIndex.java | 9 ++---
.../org/apache/ignite/internal/index/Index.java | 2 +-
.../apache/ignite/internal/index/IndexManager.java | 22 +++++-----
.../ignite/internal/index/SortedIndexImpl.java | 9 ++---
.../internal/index/event/IndexEventParameters.java | 9 ++---
.../ignite/internal/index/IndexManagerTest.java | 12 ++++--
.../ignite/internal/index/ItIndexManagerTest.java | 3 +-
.../sql/engine/ClusterPerClassIntegrationTest.java | 3 +-
.../internal/sql/engine/ItSecondaryIndexTest.java | 2 +-
.../ignite/internal/table/ItTableScanTest.java | 17 ++++----
.../index/TableIndexConfigurationSchema.java | 8 ++--
.../testutils/SchemaConfigurationConverter.java | 7 ++--
.../internal/sql/engine/schema/IgniteSchema.java | 11 +++--
.../sql/engine/schema/SqlSchemaManagerImpl.java | 15 ++++---
.../exec/rel/TableScanNodeExecutionTest.java | 3 +-
.../engine/exec/schema/SqlSchemaManagerTest.java | 3 +-
.../sql/engine/planner/AbstractPlannerTest.java | 10 ++---
.../internal/storage/engine/MvTableStorage.java | 2 +-
.../storage/index/HashIndexDescriptor.java | 16 ++++----
.../internal/storage/index/IndexDescriptor.java | 10 +++--
.../storage/index/SortedIndexDescriptor.java | 16 ++++----
.../storage/index/BinaryTupleComparatorTest.java | 10 ++---
.../storage/AbstractMvTableStorageTest.java | 8 ++--
.../internal/storage/impl/TestMvTableStorage.java | 6 +--
.../index/AbstractHashIndexStorageTest.java | 2 +-
.../index/AbstractSortedIndexStorageTest.java | 2 +-
.../pagememory/AbstractPageMemoryTableStorage.java | 2 +-
.../index/AbstractPageMemoryIndexStorage.java | 2 +-
.../storage/pagememory/index/meta/IndexMeta.java | 2 +-
.../pagememory/index/meta/IndexMetaKey.java | 8 ++--
.../pagememory/index/meta/io/IndexMetaIo.java | 34 +++++++---------
.../mv/AbstractPageMemoryMvPartitionStorage.java | 13 +++---
.../storage/rocksdb/ColumnFamilyUtils.java | 9 ++---
.../storage/rocksdb/RocksDbMetaStorage.java | 7 ++--
.../storage/rocksdb/RocksDbStorageUtils.java | 7 ++--
.../storage/rocksdb/RocksDbTableStorage.java | 12 +++---
.../rocksdb/index/AbstractRocksDbIndexStorage.java | 5 +--
.../rocksdb/index/RocksDbHashIndexStorage.java | 6 +--
.../distributed/ItTxDistributedTestSingleNode.java | 5 ++-
.../ignite/internal/table/InternalTable.java | 33 ++++++++-------
.../apache/ignite/internal/table/TableImpl.java | 40 +++++++++---------
.../table/distributed/HashIndexLocker.java | 7 ++--
.../internal/table/distributed/IndexLocker.java | 2 +-
.../table/distributed/SortedIndexLocker.java | 7 ++--
.../distributed/TableIndexStoragesSupplier.java | 5 +--
.../internal/table/distributed/TableManager.java | 22 +++-------
.../distributed/TableSchemaAwareIndexStorage.java | 7 ++--
.../distributed/command/BuildIndexCommand.java | 2 +-
.../table/distributed/index/IndexBuildTaskId.java | 11 +++--
.../table/distributed/index/IndexBuilder.java | 5 +--
.../distributed/index/IndexUpdateHandler.java | 3 +-
.../request/ScanRetrieveBatchReplicaRequest.java | 6 +--
.../replicator/PartitionReplicaListener.java | 20 ++++-----
.../distributed/storage/InternalTableImpl.java | 47 +++++++++-------------
.../internal/table/distributed/IndexBaseTest.java | 8 ++--
.../distributed/index/IndexUpdateHandlerTest.java | 3 +-
.../raft/PartitionCommandListenerTest.java | 6 +--
.../PartitionReplicaListenerIndexLockingTest.java | 19 ++++-----
.../replication/PartitionReplicaListenerTest.java | 12 +++---
.../table/impl/DummyInternalTableImpl.java | 8 ++--
61 files changed, 293 insertions(+), 348 deletions(-)
diff --git
a/modules/client/src/test/java/org/apache/ignite/client/fakes/FakeInternalTable.java
b/modules/client/src/test/java/org/apache/ignite/client/fakes/FakeInternalTable.java
index 7e800a1a96..189b495b73 100644
---
a/modules/client/src/test/java/org/apache/ignite/client/fakes/FakeInternalTable.java
+++
b/modules/client/src/test/java/org/apache/ignite/client/fakes/FakeInternalTable.java
@@ -43,7 +43,6 @@ import
org.apache.ignite.internal.util.PendingComparableValuesTracker;
import org.apache.ignite.internal.utils.PrimaryReplica;
import org.apache.ignite.lang.IgniteInternalException;
import org.apache.ignite.network.ClusterNode;
-import org.jetbrains.annotations.NotNull;
import org.jetbrains.annotations.Nullable;
/**
@@ -114,12 +113,11 @@ public class FakeInternalTable implements InternalTable {
return
CompletableFuture.completedFuture(data.get(keyExtractor.apply(keyRow).byteBuffer()));
}
- /** {@inheritDoc} */
@Override
public CompletableFuture<BinaryRow> get(
BinaryRowEx keyRow,
- @NotNull HybridTimestamp readTimestamp,
- @NotNull ClusterNode recipientNode) {
+ HybridTimestamp readTimestamp,
+ ClusterNode recipientNode) {
return null;
}
@@ -141,12 +139,11 @@ public class FakeInternalTable implements InternalTable {
return CompletableFuture.completedFuture(res);
}
- /** {@inheritDoc} */
@Override
public CompletableFuture<Collection<BinaryRow>> getAll(
Collection<BinaryRowEx> keyRows,
- @NotNull HybridTimestamp readTimestamp,
- @NotNull ClusterNode recipientNode
+ HybridTimestamp readTimestamp,
+ ClusterNode recipientNode
) {
return null;
}
@@ -332,12 +329,11 @@ public class FakeInternalTable implements InternalTable {
return CompletableFuture.completedFuture(skipped);
}
- /** {@inheritDoc} */
@Override
public Publisher<BinaryRow> scan(
int partId,
@Nullable InternalTransaction tx,
- UUID indexId,
+ @Nullable Integer indexId,
@Nullable BinaryTuplePrefix lowerBound,
@Nullable BinaryTuplePrefix upperBound,
int flags,
@@ -346,13 +342,12 @@ public class FakeInternalTable implements InternalTable {
throw new IgniteInternalException(new
OperationNotSupportedException());
}
- /** {@inheritDoc} */
@Override
public Publisher<BinaryRow> scan(
int partId,
UUID txId,
PrimaryReplica recipient,
- @Nullable UUID indexId,
+ @Nullable Integer indexId,
@Nullable BinaryTuplePrefix lowerBound,
@Nullable BinaryTuplePrefix upperBound,
int flags,
@@ -361,13 +356,12 @@ public class FakeInternalTable implements InternalTable {
throw new IgniteInternalException(new
OperationNotSupportedException());
}
- /** {@inheritDoc} */
@Override
public Publisher<BinaryRow> scan(
int partId,
- @NotNull HybridTimestamp readTimestamp,
- @NotNull ClusterNode recipientNode,
- @NotNull UUID indexId,
+ HybridTimestamp readTimestamp,
+ ClusterNode recipientNode,
+ @Nullable Integer indexId,
@Nullable BinaryTuplePrefix lowerBound,
@Nullable BinaryTuplePrefix upperBound,
int flags,
@@ -375,36 +369,33 @@ public class FakeInternalTable implements InternalTable {
throw new IgniteInternalException(new
OperationNotSupportedException());
}
- /** {@inheritDoc} */
@Override
public Publisher<BinaryRow> scan(
int partId,
- @NotNull HybridTimestamp readTimestamp,
- @NotNull ClusterNode recipientNode
+ HybridTimestamp readTimestamp,
+ ClusterNode recipientNode
) {
return null;
}
- /** {@inheritDoc} */
@Override
public Publisher<BinaryRow> lookup(
int partId,
UUID txId,
PrimaryReplica recipient,
- UUID indexId,
+ int indexId,
BinaryTuple key,
@Nullable BitSet columnsToInclude
) {
throw new IgniteInternalException(new
OperationNotSupportedException());
}
- /** {@inheritDoc} */
@Override
public Publisher<BinaryRow> lookup(
int partId,
- @NotNull HybridTimestamp readTimestamp,
- @NotNull ClusterNode recipientNode,
- @NotNull UUID indexId,
+ HybridTimestamp readTimestamp,
+ ClusterNode recipientNode,
+ int indexId,
BinaryTuple key,
@Nullable BitSet columnsToInclude
) {
diff --git
a/modules/index/src/main/java/org/apache/ignite/internal/index/HashIndex.java
b/modules/index/src/main/java/org/apache/ignite/internal/index/HashIndex.java
index 46c7dfabd1..67981fd4d8 100644
---
a/modules/index/src/main/java/org/apache/ignite/internal/index/HashIndex.java
+++
b/modules/index/src/main/java/org/apache/ignite/internal/index/HashIndex.java
@@ -34,7 +34,7 @@ import org.jetbrains.annotations.Nullable;
* An object that represents a hash index.
*/
public class HashIndex implements Index<IndexDescriptor> {
- private final UUID id;
+ private final int id;
private final InternalTable table;
private final IndexDescriptor descriptor;
@@ -45,15 +45,14 @@ public class HashIndex implements Index<IndexDescriptor> {
* @param table A table this index relates to.
* @param descriptor A descriptor of the index.
*/
- public HashIndex(UUID id, TableImpl table, IndexDescriptor descriptor) {
- this.id = Objects.requireNonNull(id, "id");
+ public HashIndex(int id, TableImpl table, IndexDescriptor descriptor) {
+ this.id = id;
this.table = Objects.requireNonNull(table.internalTable(), "table");
this.descriptor = Objects.requireNonNull(descriptor, "descriptor");
}
- /** {@inheritDoc} */
@Override
- public UUID id() {
+ public int id() {
return id;
}
diff --git
a/modules/index/src/main/java/org/apache/ignite/internal/index/Index.java
b/modules/index/src/main/java/org/apache/ignite/internal/index/Index.java
index 7d2b441979..70e606d1ba 100644
--- a/modules/index/src/main/java/org/apache/ignite/internal/index/Index.java
+++ b/modules/index/src/main/java/org/apache/ignite/internal/index/Index.java
@@ -34,7 +34,7 @@ import org.jetbrains.annotations.Nullable;
*/
public interface Index<DescriptorT extends IndexDescriptor> {
/** Returns identifier of the index. */
- UUID id();
+ int id();
/** Returns name of the index. */
String name();
diff --git
a/modules/index/src/main/java/org/apache/ignite/internal/index/IndexManager.java
b/modules/index/src/main/java/org/apache/ignite/internal/index/IndexManager.java
index 777ec8a111..443819e572 100644
---
a/modules/index/src/main/java/org/apache/ignite/internal/index/IndexManager.java
+++
b/modules/index/src/main/java/org/apache/ignite/internal/index/IndexManager.java
@@ -27,7 +27,6 @@ import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
import java.util.Objects;
-import java.util.UUID;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.function.Consumer;
@@ -48,7 +47,6 @@ import org.apache.ignite.internal.schema.Column;
import org.apache.ignite.internal.schema.SchemaDescriptor;
import org.apache.ignite.internal.schema.SchemaManager;
import org.apache.ignite.internal.schema.SchemaRegistry;
-import org.apache.ignite.internal.schema.configuration.TableConfiguration;
import org.apache.ignite.internal.schema.configuration.TableView;
import org.apache.ignite.internal.schema.configuration.TablesConfiguration;
import org.apache.ignite.internal.schema.configuration.TablesView;
@@ -203,7 +201,7 @@ public class IndexManager extends Producer<IndexEvent,
IndexEventParameters> imp
// Check index existence flag, avoid usage of hasCause +
IndexAlreadyExistsException.
AtomicBoolean idxExist = new AtomicBoolean(false);
- tablesCfg.indexes().change(indexListChange -> {
+ tablesCfg.change(tablesChange ->
tablesChange.changeIndexes(indexListChange -> {
idxExist.set(false);
if (indexListChange.get(indexName) != null) {
@@ -212,18 +210,22 @@ public class IndexManager extends Producer<IndexEvent,
IndexEventParameters> imp
throw new IndexAlreadyExistsException(schemaName,
indexName);
}
- TableConfiguration tableCfg =
tablesCfg.tables().get(tableName);
+ TableView tableCfg = tablesChange.tables().get(tableName);
if (tableCfg == null) {
throw new TableNotFoundException(schemaName, tableName);
}
- int tableId = tableCfg.id().value();
+ int tableId = tableCfg.id();
+
+ int indexId = tablesChange.globalIdCounter() + 1;
+
+ tablesChange.changeGlobalIdCounter(indexId);
- Consumer<TableIndexChange> chg = indexChange.andThen(c ->
c.changeTableId(tableId));
+ Consumer<TableIndexChange> chg = indexChange.andThen(c ->
c.changeTableId(tableId).changeId(indexId));
indexListChange.create(indexName, chg);
- }).whenComplete((index, th) -> {
+ })).whenComplete((index, th) -> {
if (th != null) {
LOG.debug("Unable to create index [schema={}, table={},
index={}]",
th, schemaName, tableName, indexName);
@@ -382,7 +384,7 @@ public class IndexManager extends Producer<IndexEvent,
IndexEventParameters> imp
private CompletableFuture<?>
onIndexDrop(ConfigurationNotificationEvent<TableIndexView> evt) {
TableIndexView tableIndexView = evt.oldValue();
- UUID idxId = tableIndexView.id();
+ int idxId = tableIndexView.id();
int tableId = tableIndexView.tableId();
@@ -423,7 +425,7 @@ public class IndexManager extends Producer<IndexEvent,
IndexEventParameters> imp
int tableId = evt.newValue().tableId();
if (!busyLock.enterBusy()) {
- UUID idxId = evt.newValue().id();
+ int idxId = evt.newValue().id();
fireEvent(IndexEvent.CREATE,
new IndexEventParameters(evt.storageRevision(), tableId,
idxId),
@@ -448,7 +450,7 @@ public class IndexManager extends Producer<IndexEvent,
IndexEventParameters> imp
) {
assert tableIndexView != null;
- UUID indexId = tableIndexView.id();
+ int indexId = tableIndexView.id();
LOG.trace("Creating local index: name={}, id={}, tableId={}, token={}",
tableIndexView.name(), indexId, tableId, causalityToken);
diff --git
a/modules/index/src/main/java/org/apache/ignite/internal/index/SortedIndexImpl.java
b/modules/index/src/main/java/org/apache/ignite/internal/index/SortedIndexImpl.java
index 59a09b3050..decdbdb593 100644
---
a/modules/index/src/main/java/org/apache/ignite/internal/index/SortedIndexImpl.java
+++
b/modules/index/src/main/java/org/apache/ignite/internal/index/SortedIndexImpl.java
@@ -35,7 +35,7 @@ import org.jetbrains.annotations.Nullable;
* An object that represents a sorted index.
*/
public class SortedIndexImpl implements SortedIndex {
- private final UUID id;
+ private final int id;
private final InternalTable table;
private final SortedIndexDescriptor descriptor;
@@ -46,15 +46,14 @@ public class SortedIndexImpl implements SortedIndex {
* @param table A table this index relates to.
* @param descriptor A descriptor of the index.
*/
- public SortedIndexImpl(UUID id, TableImpl table, SortedIndexDescriptor
descriptor) {
- this.id = Objects.requireNonNull(id, "id");
+ public SortedIndexImpl(int id, TableImpl table, SortedIndexDescriptor
descriptor) {
+ this.id = id;
this.table = Objects.requireNonNull(table.internalTable(), "table");
this.descriptor = Objects.requireNonNull(descriptor, "descriptor");
}
- /** {@inheritDoc} */
@Override
- public UUID id() {
+ public int id() {
return id;
}
diff --git
a/modules/index/src/main/java/org/apache/ignite/internal/index/event/IndexEventParameters.java
b/modules/index/src/main/java/org/apache/ignite/internal/index/event/IndexEventParameters.java
index 4319f6c9a0..222cd0d566 100644
---
a/modules/index/src/main/java/org/apache/ignite/internal/index/event/IndexEventParameters.java
+++
b/modules/index/src/main/java/org/apache/ignite/internal/index/event/IndexEventParameters.java
@@ -17,7 +17,6 @@
package org.apache.ignite.internal.index.event;
-import java.util.UUID;
import org.apache.ignite.internal.index.IndexDescriptor;
import org.apache.ignite.internal.manager.EventParameters;
import org.jetbrains.annotations.Nullable;
@@ -30,7 +29,7 @@ public class IndexEventParameters extends EventParameters {
private final int tableId;
/** Index identifier. */
- private final UUID indexId;
+ private final int indexId;
/** Index instance. */
private final @Nullable IndexDescriptor indexDescriptor;
@@ -42,7 +41,7 @@ public class IndexEventParameters extends EventParameters {
* @param tableId Table identifier.
* @param indexId Index identifier.
*/
- public IndexEventParameters(long revision, int tableId, UUID indexId) {
+ public IndexEventParameters(long revision, int tableId, int indexId) {
this(revision, tableId, indexId, null);
}
@@ -54,7 +53,7 @@ public class IndexEventParameters extends EventParameters {
* @param indexId Index identifier.
* @param indexDescriptor Index descriptor.
*/
- public IndexEventParameters(long revision, int tableId, UUID indexId,
@Nullable IndexDescriptor indexDescriptor) {
+ public IndexEventParameters(long revision, int tableId, int indexId,
@Nullable IndexDescriptor indexDescriptor) {
super(revision);
this.tableId = tableId;
@@ -76,7 +75,7 @@ public class IndexEventParameters extends EventParameters {
*
* @return An id of the index.
*/
- public UUID indexId() {
+ public int indexId() {
return indexId;
}
diff --git
a/modules/index/src/test/java/org/apache/ignite/internal/index/IndexManagerTest.java
b/modules/index/src/test/java/org/apache/ignite/internal/index/IndexManagerTest.java
index c24cac3d3e..aed8ba145a 100644
---
a/modules/index/src/test/java/org/apache/ignite/internal/index/IndexManagerTest.java
+++
b/modules/index/src/test/java/org/apache/ignite/internal/index/IndexManagerTest.java
@@ -18,6 +18,7 @@
package org.apache.ignite.internal.index;
import static java.util.concurrent.CompletableFuture.completedFuture;
+import static java.util.stream.Collectors.toList;
import static
org.apache.ignite.internal.testframework.matchers.CompletableFutureExceptionMatcher.willThrowFast;
import static
org.apache.ignite.internal.testframework.matchers.CompletableFutureMatcher.willCompleteSuccessfully;
import static org.apache.ignite.lang.IgniteStringFormatter.format;
@@ -35,7 +36,6 @@ import java.util.ArrayList;
import java.util.List;
import java.util.Map;
import java.util.Objects;
-import java.util.UUID;
import java.util.concurrent.atomic.AtomicReference;
import
org.apache.ignite.internal.configuration.testframework.ConfigurationExtension;
import
org.apache.ignite.internal.configuration.testframework.InjectConfiguration;
@@ -47,6 +47,7 @@ import org.apache.ignite.internal.schema.SchemaManager;
import org.apache.ignite.internal.schema.configuration.ExtendedTableChange;
import org.apache.ignite.internal.schema.configuration.TablesConfiguration;
import org.apache.ignite.internal.schema.configuration.index.SortedIndexChange;
+import org.apache.ignite.internal.schema.configuration.index.TableIndexView;
import org.apache.ignite.internal.table.InternalTable;
import org.apache.ignite.internal.table.TableImpl;
import org.apache.ignite.internal.table.distributed.TableManager;
@@ -139,7 +140,8 @@ public class IndexManagerTest {
"name", indexName,
"type", "SORTED",
"uniq", false,
- "tableId", tableId()
+ "tableId", tableId(),
+ "id", 1
)
);
@@ -191,11 +193,13 @@ public class IndexManagerTest {
sortedIndexChange.changeTableId(tableId());
}), willCompleteSuccessfully());
- List<UUID> indexIds = tablesConfig.indexes().internalIds();
+ List<Integer> indexIds = tablesConfig.indexes().value().stream()
+ .map(TableIndexView::id)
+ .collect(toList());
assertThat(indexIds, hasSize(1));
- UUID indexId = indexIds.get(0);
+ int indexId = indexIds.get(0);
assertThat(holder.get(), notNullValue());
assertThat(holder.get().indexId(), equalTo(indexId));
diff --git
a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/index/ItIndexManagerTest.java
b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/index/ItIndexManagerTest.java
index c63898cd90..e511cac7c0 100644
---
a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/index/ItIndexManagerTest.java
+++
b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/index/ItIndexManagerTest.java
@@ -23,7 +23,6 @@ import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.hasItems;
import static org.hamcrest.Matchers.notNullValue;
-import java.util.UUID;
import java.util.concurrent.CompletableFuture;
import org.apache.ignite.Ignite;
import org.apache.ignite.internal.app.IgniteImpl;
@@ -77,7 +76,7 @@ public class ItIndexManagerTest extends
ClusterPerClassIntegrationTest {
tableIndexChange ->
tableIndexChange.convert(HashIndexChange.class).changeColumnNames("C3", "C2")
));
- UUID createdIndexId;
+ int createdIndexId;
{
IndexEventParameters parameters = await(indexCreatedFuture);
diff --git
a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ClusterPerClassIntegrationTest.java
b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ClusterPerClassIntegrationTest.java
index 7f641fdd75..e10924fb03 100644
---
a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ClusterPerClassIntegrationTest.java
+++
b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ClusterPerClassIntegrationTest.java
@@ -38,7 +38,6 @@ import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Objects;
-import java.util.UUID;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.TimeUnit;
import java.util.stream.IntStream;
@@ -516,7 +515,7 @@ public abstract class ClusterPerClassIntegrationTest
extends IgniteIntegrationTe
TablesView tablesView =
getTablesConfiguration(clusterNode).value();
- UUID indexId =
tablesView.indexes().get(indexName.toUpperCase()).id();
+ int indexId =
tablesView.indexes().get(indexName.toUpperCase()).id();
IndexStorage index =
internalTable.storage().getOrCreateIndex(partitionId,
createIndexDescriptor(tablesView, indexId));
diff --git
a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItSecondaryIndexTest.java
b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItSecondaryIndexTest.java
index 4ddf21cad0..6a97d53c9d 100644
---
a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItSecondaryIndexTest.java
+++
b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItSecondaryIndexTest.java
@@ -1065,7 +1065,7 @@ public class ItSecondaryIndexTest extends
ClusterPerClassIntegrationTest {
}
@Override
- public UUID id() {
+ public int id() {
return delegate.id();
}
diff --git
a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/table/ItTableScanTest.java
b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/table/ItTableScanTest.java
index a81152dfa1..9c2ba43250 100644
---
a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/table/ItTableScanTest.java
+++
b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/table/ItTableScanTest.java
@@ -34,7 +34,6 @@ import static org.junit.jupiter.api.Assertions.assertTrue;
import java.util.ArrayList;
import java.util.List;
-import java.util.UUID;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.Flow.Publisher;
import java.util.concurrent.Flow.Subscriber;
@@ -135,7 +134,7 @@ public class ItTableScanTest extends
ClusterPerClassIntegrationTest {
InternalTransaction tx0 = (InternalTransaction) transactions.begin();
InternalTransaction tx1 = startTxWithEnlistedPartition(PART_ID, false);
- UUID sortedIndexId = getSortedIndexId();
+ int sortedIndexId = getSortedIndexId();
List<BinaryRow> scannedRows = new ArrayList<>();
@@ -174,7 +173,7 @@ public class ItTableScanTest extends
ClusterPerClassIntegrationTest {
@Test
public void testInsertDuringScan() throws Exception {
- UUID sortedIndexId = getSortedIndexId();
+ int sortedIndexId = getSortedIndexId();
List<BinaryRow> scannedRows = new ArrayList<>();
@@ -399,7 +398,7 @@ public class ItTableScanTest extends
ClusterPerClassIntegrationTest {
public void testTwiceScanInTransaction() throws Exception {
KeyValueView<Tuple, Tuple> kvView = table.keyValueView();
- UUID sortedIndexId = getSortedIndexId();
+ int sortedIndexId = getSortedIndexId();
List<BinaryRow> scannedRows = new ArrayList<>();
@@ -454,7 +453,7 @@ public class ItTableScanTest extends
ClusterPerClassIntegrationTest {
BinaryTuplePrefix upperBound = BinaryTuplePrefix.fromBinaryTuple(new
BinaryTuple(sortedIndexBinarySchema,
new BinaryTupleBuilder(1, false).appendInt(9).build()));
- UUID soredIndexId = getSortedIndexId();
+ int soredIndexId = getSortedIndexId();
InternalTransaction tx = startTxWithEnlistedPartition(PART_ID, false);
PrimaryReplica recipient = getLeaderRecipient(PART_ID, tx);
@@ -529,7 +528,7 @@ public class ItTableScanTest extends
ClusterPerClassIntegrationTest {
for (int i = 0; i < iterations; i++) {
KeyValueView<Tuple, Tuple> kvView = table.keyValueView();
- UUID sortedIndexId = getSortedIndexId();
+ int sortedIndexId = getSortedIndexId();
InternalTransaction tx = startTxWithEnlistedPartition(PART_ID,
false);
@@ -611,7 +610,7 @@ public class ItTableScanTest extends
ClusterPerClassIntegrationTest {
kvView.remove(null, Tuple.create().set("key", ROW_IDS.get(1)));
kvView.put(null, Tuple.create().set("key", ROW_IDS.get(2)),
Tuple.create().set("valInt", 999).set("valStr", "Str_999"));
- UUID sortedIndexId = getSortedIndexId();
+ int sortedIndexId = getSortedIndexId();
InternalTransaction tx = startTxWithEnlistedPartition(PART_ID,
readOnly);
@@ -713,10 +712,8 @@ public class ItTableScanTest extends
ClusterPerClassIntegrationTest {
/**
* Gets an index id.
- *
- * @return Index id.
*/
- private static UUID getSortedIndexId() {
+ private static int getSortedIndexId() {
return getSortedIndexConfig(CLUSTER_NODES.get(0)).id().value();
}
diff --git
a/modules/schema/src/main/java/org/apache/ignite/internal/schema/configuration/index/TableIndexConfigurationSchema.java
b/modules/schema/src/main/java/org/apache/ignite/internal/schema/configuration/index/TableIndexConfigurationSchema.java
index 6cbd0412b6..0e1c6d54ce 100644
---
a/modules/schema/src/main/java/org/apache/ignite/internal/schema/configuration/index/TableIndexConfigurationSchema.java
+++
b/modules/schema/src/main/java/org/apache/ignite/internal/schema/configuration/index/TableIndexConfigurationSchema.java
@@ -17,12 +17,11 @@
package org.apache.ignite.internal.schema.configuration.index;
-import java.util.UUID;
import org.apache.ignite.configuration.annotation.InjectedName;
-import org.apache.ignite.configuration.annotation.InternalId;
import org.apache.ignite.configuration.annotation.PolymorphicConfig;
import org.apache.ignite.configuration.annotation.PolymorphicId;
import org.apache.ignite.configuration.annotation.Value;
+import org.apache.ignite.configuration.validation.Range;
/**
* SQL index configuration.
@@ -40,8 +39,9 @@ public class TableIndexConfigurationSchema {
public String type;
/** Index identifier. */
- @InternalId
- public UUID id;
+ @Value(hasDefault = true)
+ @Range(min = 1)
+ public int id = 1;
/** Index name. */
@InjectedName
diff --git
a/modules/schema/src/testFixtures/java/org/apache/ignite/internal/schema/testutils/SchemaConfigurationConverter.java
b/modules/schema/src/testFixtures/java/org/apache/ignite/internal/schema/testutils/SchemaConfigurationConverter.java
index f7ec77867f..1004901a2f 100644
---
a/modules/schema/src/testFixtures/java/org/apache/ignite/internal/schema/testutils/SchemaConfigurationConverter.java
+++
b/modules/schema/src/testFixtures/java/org/apache/ignite/internal/schema/testutils/SchemaConfigurationConverter.java
@@ -484,12 +484,13 @@ public class SchemaConfigurationConverter {
* Add index.
*
* @param idx Index to add.
- * @param tableId Table id.
+ * @param tableId Table ID.
+ * @param indexId Index ID.
* @param change Indexes change to fulfill.
* @return TableChange to get result from.
*/
- public static TableIndexChange addIndex(IndexDefinition idx, int tableId,
TableIndexChange change) {
- return convert(idx, change).changeTableId(tableId);
+ public static TableIndexChange addIndex(IndexDefinition idx, int tableId,
int indexId, TableIndexChange change) {
+ return convert(idx, change).changeTableId(tableId).changeId(indexId);
}
/**
diff --git
a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/schema/IgniteSchema.java
b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/schema/IgniteSchema.java
index f9012fa287..9aa3ef3b44 100644
---
a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/schema/IgniteSchema.java
+++
b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/schema/IgniteSchema.java
@@ -19,7 +19,6 @@ package org.apache.ignite.internal.sql.engine.schema;
import java.util.Collections;
import java.util.Map;
-import java.util.UUID;
import java.util.concurrent.ConcurrentHashMap;
import org.apache.calcite.schema.Table;
import org.apache.calcite.schema.impl.AbstractSchema;
@@ -35,7 +34,7 @@ public class IgniteSchema extends AbstractSchema {
private final Map<String, Table> tblMap;
- private final Map<UUID, IgniteIndex> idxMap;
+ private final Map<Integer, IgniteIndex> idxMap;
private final long schemaVersion;
@@ -49,7 +48,7 @@ public class IgniteSchema extends AbstractSchema {
public IgniteSchema(
String schemaName,
@Nullable Map<String, Table> tableMap,
- @Nullable Map<UUID, IgniteIndex> indexMap,
+ @Nullable Map<Integer, IgniteIndex> indexMap,
long schemaVersion
) {
this.schemaName = schemaName;
@@ -119,7 +118,7 @@ public class IgniteSchema extends AbstractSchema {
* @param indexId Index id.
* @param index Index.
*/
- public void addIndex(UUID indexId, IgniteIndex index) {
+ public void addIndex(int indexId, IgniteIndex index) {
idxMap.put(indexId, index);
}
@@ -129,7 +128,7 @@ public class IgniteSchema extends AbstractSchema {
* @param indexId Index id.
* @return Removed index.
*/
- public IgniteIndex removeIndex(UUID indexId) {
+ public IgniteIndex removeIndex(int indexId) {
return idxMap.remove(indexId);
}
@@ -139,7 +138,7 @@ public class IgniteSchema extends AbstractSchema {
* @param indexId Index id.
* @return Index.
*/
- public IgniteIndex index(UUID indexId) {
+ public IgniteIndex index(int indexId) {
return idxMap.get(indexId);
}
diff --git
a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/schema/SqlSchemaManagerImpl.java
b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/schema/SqlSchemaManagerImpl.java
index 00bf72aa05..0f4374a3b8 100644
---
a/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/schema/SqlSchemaManagerImpl.java
+++
b/modules/sql-engine/src/main/java/org/apache/ignite/internal/sql/engine/schema/SqlSchemaManagerImpl.java
@@ -33,7 +33,6 @@ import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Set;
-import java.util.UUID;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.CopyOnWriteArraySet;
@@ -77,7 +76,7 @@ public class SqlSchemaManagerImpl implements SqlSchemaManager
{
private final Map<Integer, CompletableFuture<?>> pkIdxReady = new
ConcurrentHashMap<>();
- private final IncrementalVersionedValue<Map<UUID, IgniteIndex>> indicesVv;
+ private final IncrementalVersionedValue<Map<Integer, IgniteIndex>>
indicesVv;
private final TableManager tableManager;
private final SchemaManager schemaManager;
@@ -412,13 +411,13 @@ public class SqlSchemaManagerImpl implements
SqlSchemaManager {
* @param causalityToken Causality token.
* @return Schema registration future.
*/
- public CompletableFuture<?> onIndexCreated(int tableId, UUID indexId,
IndexDescriptor indexDescriptor, long causalityToken) {
+ public CompletableFuture<?> onIndexCreated(int tableId, int indexId,
IndexDescriptor indexDescriptor, long causalityToken) {
if (!busyLock.enterBusy()) {
return failedFuture(new IgniteInternalException(NODE_STOPPING_ERR,
new NodeStoppingException()));
}
try {
- CompletableFuture<Map<UUID, IgniteIndex>> updatedIndices =
indicesVv.update(causalityToken, (indices, e) ->
+ CompletableFuture<Map<Integer, IgniteIndex>> updatedIndices =
indicesVv.update(causalityToken, (indices, e) ->
inBusyLock(busyLock, () -> {
if (e != null) {
return failedFuture(e);
@@ -427,7 +426,7 @@ public class SqlSchemaManagerImpl implements
SqlSchemaManager {
return tableManager.tableAsync(causalityToken,
tableId).thenApply(table -> {
var igniteIndex = new IgniteIndex(newIndex(table,
indexId, indexDescriptor));
- Map<UUID, IgniteIndex> resIdxs = new
HashMap<>(indices);
+ Map<Integer, IgniteIndex> resIdxs = new
HashMap<>(indices);
resIdxs.put(indexId, igniteIndex);
@@ -491,7 +490,7 @@ public class SqlSchemaManagerImpl implements
SqlSchemaManager {
}
}
- private static Index<?> newIndex(TableImpl table, UUID indexId,
IndexDescriptor descriptor) {
+ private static Index<?> newIndex(TableImpl table, int indexId,
IndexDescriptor descriptor) {
if (descriptor instanceof SortedIndexDescriptor) {
return new SortedIndexImpl(indexId, table, (SortedIndexDescriptor)
descriptor);
} else {
@@ -507,7 +506,7 @@ public class SqlSchemaManagerImpl implements
SqlSchemaManager {
* @param causalityToken Causality token.
* @return Schema registration future.
*/
- public CompletableFuture<?> onIndexDropped(String schemaName, int tableId,
UUID indexId, long causalityToken) {
+ public CompletableFuture<?> onIndexDropped(String schemaName, int tableId,
int indexId, long causalityToken) {
if (!busyLock.enterBusy()) {
return failedFuture(new IgniteInternalException(NODE_STOPPING_ERR,
new NodeStoppingException()));
}
@@ -522,7 +521,7 @@ public class SqlSchemaManagerImpl implements
SqlSchemaManager {
return failedFuture(e);
}
- Map<UUID, IgniteIndex> resIdxs = new HashMap<>(indices);
+ Map<Integer, IgniteIndex> resIdxs = new HashMap<>(indices);
IgniteIndex rmvIdx = resIdxs.remove(indexId);
diff --git
a/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/exec/rel/TableScanNodeExecutionTest.java
b/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/exec/rel/TableScanNodeExecutionTest.java
index 20837d2995..5875355925 100644
---
a/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/exec/rel/TableScanNodeExecutionTest.java
+++
b/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/exec/rel/TableScanNodeExecutionTest.java
@@ -23,7 +23,6 @@ import static org.mockito.Mockito.mock;
import it.unimi.dsi.fastutil.ints.Int2ObjectMaps;
import java.util.BitSet;
import java.util.List;
-import java.util.UUID;
import java.util.concurrent.Flow.Publisher;
import java.util.concurrent.Flow.Subscription;
import java.util.concurrent.ThreadLocalRandom;
@@ -169,7 +168,7 @@ public class TableScanNodeExecutionTest extends
AbstractExecutionTest {
int partId,
HybridTimestamp readTime,
ClusterNode recipient,
- @Nullable UUID indexId,
+ @Nullable Integer indexId,
@Nullable BinaryTuplePrefix lowerBound,
@Nullable BinaryTuplePrefix upperBound,
int flags,
diff --git
a/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/exec/schema/SqlSchemaManagerTest.java
b/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/exec/schema/SqlSchemaManagerTest.java
index d4fa19a9af..a27e78baeb 100644
---
a/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/exec/schema/SqlSchemaManagerTest.java
+++
b/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/exec/schema/SqlSchemaManagerTest.java
@@ -34,7 +34,6 @@ import static org.mockito.Mockito.verifyNoMoreInteractions;
import static org.mockito.Mockito.when;
import java.util.List;
-import java.util.UUID;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.atomic.AtomicLong;
import java.util.function.Consumer;
@@ -74,7 +73,7 @@ import org.mockito.junit.jupiter.MockitoExtension;
public class SqlSchemaManagerTest {
private final int tableId = 1;
- private final UUID indexId = UUID.randomUUID();
+ private final int indexId = 2;
private final SchemaDescriptor schemaDescriptor = new SchemaDescriptor(
1,
diff --git
a/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/planner/AbstractPlannerTest.java
b/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/planner/AbstractPlannerTest.java
index b8f152ec48..5d93c09ceb 100644
---
a/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/planner/AbstractPlannerTest.java
+++
b/modules/sql-engine/src/test/java/org/apache/ignite/internal/sql/engine/planner/AbstractPlannerTest.java
@@ -1207,7 +1207,7 @@ public abstract class AbstractPlannerTest extends
IgniteAbstractTest {
}
static class TestSortedIndex implements SortedIndex {
- private final UUID id = UUID.randomUUID();
+ private final int id = 1;
private final int tableId = 1;
@@ -1235,9 +1235,8 @@ public abstract class AbstractPlannerTest extends
IgniteAbstractTest {
this.descriptor = descriptor;
}
- /** {@inheritDoc} */
@Override
- public UUID id() {
+ public int id() {
return id;
}
@@ -1288,7 +1287,7 @@ public abstract class AbstractPlannerTest extends
IgniteAbstractTest {
/** Test Hash index implementation. */
public static class TestHashIndex implements Index<IndexDescriptor> {
- private final UUID id = UUID.randomUUID();
+ private final int id = 1;
private int tableId = 1;
@@ -1316,9 +1315,8 @@ public abstract class AbstractPlannerTest extends
IgniteAbstractTest {
this.descriptor = descriptor;
}
- /** {@inheritDoc} */
@Override
- public UUID id() {
+ public int id() {
return id;
}
diff --git
a/modules/storage-api/src/main/java/org/apache/ignite/internal/storage/engine/MvTableStorage.java
b/modules/storage-api/src/main/java/org/apache/ignite/internal/storage/engine/MvTableStorage.java
index bce10d3899..b6a2a3b866 100644
---
a/modules/storage-api/src/main/java/org/apache/ignite/internal/storage/engine/MvTableStorage.java
+++
b/modules/storage-api/src/main/java/org/apache/ignite/internal/storage/engine/MvTableStorage.java
@@ -125,7 +125,7 @@ public interface MvTableStorage extends ManuallyCloseable {
*
* @param indexId Index ID.
*/
- CompletableFuture<Void> destroyIndex(UUID indexId);
+ CompletableFuture<Void> destroyIndex(int indexId);
/**
* Returns {@code true} if this storage is volatile (i.e. stores its data
in memory), or {@code false} if it's persistent.
diff --git
a/modules/storage-api/src/main/java/org/apache/ignite/internal/storage/index/HashIndexDescriptor.java
b/modules/storage-api/src/main/java/org/apache/ignite/internal/storage/index/HashIndexDescriptor.java
index 284dca252c..e93a8e9088 100644
---
a/modules/storage-api/src/main/java/org/apache/ignite/internal/storage/index/HashIndexDescriptor.java
+++
b/modules/storage-api/src/main/java/org/apache/ignite/internal/storage/index/HashIndexDescriptor.java
@@ -21,7 +21,6 @@ import static java.util.stream.Collectors.toUnmodifiableList;
import java.util.Arrays;
import java.util.List;
-import java.util.UUID;
import org.apache.ignite.configuration.NamedListView;
import org.apache.ignite.internal.schema.NativeType;
import org.apache.ignite.internal.schema.configuration.ColumnView;
@@ -95,7 +94,7 @@ public class HashIndexDescriptor implements IndexDescriptor {
}
}
- private final UUID id;
+ private final int id;
private final List<HashIndexColumnDescriptor> columns;
@@ -105,7 +104,7 @@ public class HashIndexDescriptor implements IndexDescriptor
{
* @param indexId Index id.
* @param tablesConfig Tables and indexes configuration.
*/
- public HashIndexDescriptor(UUID indexId, TablesView tablesConfig) {
+ public HashIndexDescriptor(int indexId, TablesView tablesConfig) {
this(indexId, extractIndexColumnsConfiguration(indexId, tablesConfig));
}
@@ -115,13 +114,16 @@ public class HashIndexDescriptor implements
IndexDescriptor {
* @param indexId Index id.
* @param columns Columns descriptors.
*/
- public HashIndexDescriptor(UUID indexId, List<HashIndexColumnDescriptor>
columns) {
+ public HashIndexDescriptor(int indexId, List<HashIndexColumnDescriptor>
columns) {
this.id = indexId;
this.columns = columns;
}
- private static List<HashIndexColumnDescriptor>
extractIndexColumnsConfiguration(UUID indexId, TablesView tablesConfig) {
- TableIndexView indexConfig = tablesConfig.indexes().get(indexId);
+ private static List<HashIndexColumnDescriptor>
extractIndexColumnsConfiguration(int indexId, TablesView tablesConfig) {
+ TableIndexView indexConfig = tablesConfig.indexes().stream()
+ .filter(tableIndexView -> tableIndexView.id() == indexId)
+ .findFirst()
+ .orElse(null);
if (indexConfig == null) {
throw new StorageException(String.format("Index configuration for
\"%s\" could not be found", indexId));
@@ -164,7 +166,7 @@ public class HashIndexDescriptor implements IndexDescriptor
{
}
@Override
- public UUID id() {
+ public int id() {
return id;
}
diff --git
a/modules/storage-api/src/main/java/org/apache/ignite/internal/storage/index/IndexDescriptor.java
b/modules/storage-api/src/main/java/org/apache/ignite/internal/storage/index/IndexDescriptor.java
index d3a0743f34..1500e096b6 100644
---
a/modules/storage-api/src/main/java/org/apache/ignite/internal/storage/index/IndexDescriptor.java
+++
b/modules/storage-api/src/main/java/org/apache/ignite/internal/storage/index/IndexDescriptor.java
@@ -18,7 +18,6 @@
package org.apache.ignite.internal.storage.index;
import java.util.List;
-import java.util.UUID;
import org.apache.ignite.internal.schema.NativeType;
import org.apache.ignite.internal.schema.configuration.TablesView;
import org.apache.ignite.internal.schema.configuration.index.HashIndexView;
@@ -52,7 +51,7 @@ public interface IndexDescriptor {
/**
* Returns the index ID.
*/
- UUID id();
+ int id();
/**
* Returns index column descriptions.
@@ -65,8 +64,11 @@ public interface IndexDescriptor {
* @param tablesView Tables configuration.
* @param indexId Index ID.
*/
- static IndexDescriptor createIndexDescriptor(TablesView tablesView, UUID
indexId) {
- TableIndexView indexView = tablesView.indexes().get(indexId);
+ static IndexDescriptor createIndexDescriptor(TablesView tablesView, int
indexId) {
+ TableIndexView indexView = tablesView.indexes().stream()
+ .filter(tableIndexView -> indexId == tableIndexView.id())
+ .findFirst()
+ .orElse(null);
if (indexView instanceof HashIndexView) {
return new HashIndexDescriptor(indexId, tablesView);
diff --git
a/modules/storage-api/src/main/java/org/apache/ignite/internal/storage/index/SortedIndexDescriptor.java
b/modules/storage-api/src/main/java/org/apache/ignite/internal/storage/index/SortedIndexDescriptor.java
index 1e22bab27a..2b28e21e66 100644
---
a/modules/storage-api/src/main/java/org/apache/ignite/internal/storage/index/SortedIndexDescriptor.java
+++
b/modules/storage-api/src/main/java/org/apache/ignite/internal/storage/index/SortedIndexDescriptor.java
@@ -20,7 +20,6 @@ package org.apache.ignite.internal.storage.index;
import static java.util.stream.Collectors.toUnmodifiableList;
import java.util.List;
-import java.util.UUID;
import org.apache.ignite.configuration.NamedListView;
import org.apache.ignite.internal.schema.BinaryTupleSchema;
import org.apache.ignite.internal.schema.BinaryTupleSchema.Element;
@@ -110,7 +109,7 @@ public class SortedIndexDescriptor implements
IndexDescriptor {
}
}
- private final UUID id;
+ private final int id;
private final List<SortedIndexColumnDescriptor> columns;
@@ -122,7 +121,7 @@ public class SortedIndexDescriptor implements
IndexDescriptor {
* @param indexId Index ID.
* @param tablesConfig Tables configuration.
*/
- public SortedIndexDescriptor(UUID indexId, TablesView tablesConfig) {
+ public SortedIndexDescriptor(int indexId, TablesView tablesConfig) {
this(indexId, extractIndexColumnsConfiguration(indexId, tablesConfig));
}
@@ -132,14 +131,17 @@ public class SortedIndexDescriptor implements
IndexDescriptor {
* @param indexId Index ID.
* @param columnDescriptors Column descriptors.
*/
- public SortedIndexDescriptor(UUID indexId,
List<SortedIndexColumnDescriptor> columnDescriptors) {
+ public SortedIndexDescriptor(int indexId,
List<SortedIndexColumnDescriptor> columnDescriptors) {
this.id = indexId;
this.columns = List.copyOf(columnDescriptors);
this.binaryTupleSchema = createSchema(columns);
}
- private static List<SortedIndexColumnDescriptor>
extractIndexColumnsConfiguration(UUID indexId, TablesView tablesConfig) {
- TableIndexView indexConfig = tablesConfig.indexes().get(indexId);
+ private static List<SortedIndexColumnDescriptor>
extractIndexColumnsConfiguration(int indexId, TablesView tablesConfig) {
+ TableIndexView indexConfig = tablesConfig.indexes().stream()
+ .filter(tableIndexView -> tableIndexView.id() == indexId)
+ .findFirst()
+ .orElse(null);
if (indexConfig == null) {
throw new StorageException(String.format("Index configuration for
\"%s\" could not be found", indexId));
@@ -192,7 +194,7 @@ public class SortedIndexDescriptor implements
IndexDescriptor {
}
@Override
- public UUID id() {
+ public int id() {
return id;
}
diff --git
a/modules/storage-api/src/test/java/org/apache/ignite/internal/storage/index/BinaryTupleComparatorTest.java
b/modules/storage-api/src/test/java/org/apache/ignite/internal/storage/index/BinaryTupleComparatorTest.java
index e527622cd6..c1a848abd4 100644
---
a/modules/storage-api/src/test/java/org/apache/ignite/internal/storage/index/BinaryTupleComparatorTest.java
+++
b/modules/storage-api/src/test/java/org/apache/ignite/internal/storage/index/BinaryTupleComparatorTest.java
@@ -53,7 +53,7 @@ public class BinaryTupleComparatorTest {
public void testCompareSingleColumnTuples(NativeType type) {
var columnDescriptor = new SortedIndexColumnDescriptor("column", type,
false, true);
- var descriptor = new SortedIndexDescriptor(UUID.randomUUID(),
List.of(columnDescriptor));
+ var descriptor = new SortedIndexDescriptor(1,
List.of(columnDescriptor));
var comparator = new BinaryTupleComparator(descriptor);
@@ -305,7 +305,7 @@ public class BinaryTupleComparatorTest {
new SortedIndexColumnDescriptor("column", NativeTypes.STRING,
false, false)
);
- var descriptor = new SortedIndexDescriptor(UUID.randomUUID(),
columnDescriptors);
+ var descriptor = new SortedIndexDescriptor(1, columnDescriptors);
var comparator = new BinaryTupleComparator(descriptor);
@@ -339,7 +339,7 @@ public class BinaryTupleComparatorTest {
new SortedIndexColumnDescriptor("column", NativeTypes.STRING,
true, false)
);
- var descriptor = new SortedIndexDescriptor(UUID.randomUUID(),
columnDescriptors);
+ var descriptor = new SortedIndexDescriptor(1, columnDescriptors);
var comparator = new BinaryTupleComparator(descriptor);
@@ -382,7 +382,7 @@ public class BinaryTupleComparatorTest {
new SortedIndexColumnDescriptor("column", NativeTypes.STRING,
false, false)
);
- var descriptor = new SortedIndexDescriptor(UUID.randomUUID(),
columnDescriptors);
+ var descriptor = new SortedIndexDescriptor(1, columnDescriptors);
var comparator = new BinaryTupleComparator(descriptor);
@@ -425,7 +425,7 @@ public class BinaryTupleComparatorTest {
new SortedIndexColumnDescriptor("column", NativeTypes.STRING,
false, false)
);
- var descriptor = new SortedIndexDescriptor(UUID.randomUUID(),
columnDescriptors);
+ var descriptor = new SortedIndexDescriptor(1, columnDescriptors);
var comparator = new BinaryTupleComparator(descriptor);
diff --git
a/modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/AbstractMvTableStorageTest.java
b/modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/AbstractMvTableStorageTest.java
index 656ea9fda6..7981458b24 100644
---
a/modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/AbstractMvTableStorageTest.java
+++
b/modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/AbstractMvTableStorageTest.java
@@ -343,16 +343,16 @@ public abstract class AbstractMvTableStorageTest extends
BaseMvStoragesTest {
tableStorage.createMvPartition(PARTITION_ID);
- UUID invalidUuid = UUID.randomUUID();
+ int invalidId = Integer.MAX_VALUE;
TablesView tablesView = tableStorage.tablesConfiguration().value();
e = assertThrows(
StorageException.class,
- () -> new HashIndexDescriptor(invalidUuid, tablesView)
+ () -> new HashIndexDescriptor(invalidId, tablesView)
);
- assertThat(e.getMessage(), containsString(String.format("Index
configuration for \"%s\" could not be found", invalidUuid)));
+ assertThat(e.getMessage(), containsString(String.format("Index
configuration for \"%s\" could not be found", invalidId)));
e = assertThrows(
StorageException.class,
@@ -833,7 +833,7 @@ public abstract class AbstractMvTableStorageTest extends
BaseMvStoragesTest {
CompletableFuture<Void> indexCreateFut =
tablesConfig.indexes().change(ch ->
indexDefinitions.forEach(idxDef -> ch.create(idxDef.name(),
- c -> SchemaConfigurationConverter.addIndex(idxDef,
tableId, c)
+ c -> SchemaConfigurationConverter.addIndex(idxDef,
tableId, idxDef.name().hashCode(), c)
))
);
diff --git
a/modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/impl/TestMvTableStorage.java
b/modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/impl/TestMvTableStorage.java
index 7affb0b0f2..b9b1f36ae7 100644
---
a/modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/impl/TestMvTableStorage.java
+++
b/modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/impl/TestMvTableStorage.java
@@ -51,9 +51,9 @@ import org.jetbrains.annotations.Nullable;
public class TestMvTableStorage implements MvTableStorage {
private final MvPartitionStorages<TestMvPartitionStorage>
mvPartitionStorages;
- private final Map<UUID, SortedIndices> sortedIndicesById = new
ConcurrentHashMap<>();
+ private final Map<Integer, SortedIndices> sortedIndicesById = new
ConcurrentHashMap<>();
- private final Map<UUID, HashIndices> hashIndicesById = new
ConcurrentHashMap<>();
+ private final Map<Integer, HashIndices> hashIndicesById = new
ConcurrentHashMap<>();
private final TableConfiguration tableCfg;
@@ -175,7 +175,7 @@ public class TestMvTableStorage implements MvTableStorage {
}
@Override
- public CompletableFuture<Void> destroyIndex(UUID indexId) {
+ public CompletableFuture<Void> destroyIndex(int indexId) {
sortedIndicesById.remove(indexId);
HashIndices hashIndex = hashIndicesById.remove(indexId);
diff --git
a/modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/index/AbstractHashIndexStorageTest.java
b/modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/index/AbstractHashIndexStorageTest.java
index e3c5bedf27..f6ae7e5006 100644
---
a/modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/index/AbstractHashIndexStorageTest.java
+++
b/modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/index/AbstractHashIndexStorageTest.java
@@ -48,7 +48,7 @@ public abstract class AbstractHashIndexStorageTest extends
AbstractIndexStorageT
.change(chg -> chg.create(indexDefinition.name(), idx -> {
int tableId =
tablesCfg.tables().value().get(TABLE_NAME).id();
- addIndex(indexDefinition, tableId, idx);
+ addIndex(indexDefinition, tableId,
indexDefinition.name().hashCode(), idx);
}));
assertThat(createIndexFuture, willCompleteSuccessfully());
diff --git
a/modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/index/AbstractSortedIndexStorageTest.java
b/modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/index/AbstractSortedIndexStorageTest.java
index cd7a3a8078..1c5982167e 100644
---
a/modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/index/AbstractSortedIndexStorageTest.java
+++
b/modules/storage-api/src/testFixtures/java/org/apache/ignite/internal/storage/index/AbstractSortedIndexStorageTest.java
@@ -122,7 +122,7 @@ public abstract class AbstractSortedIndexStorageTest
extends AbstractIndexStorag
tablesCfg.indexes().change(chg ->
chg.create(indexDefinition.name(), idx -> {
int tableId =
tablesCfg.tables().value().get(TABLE_NAME).id();
- addIndex(indexDefinition, tableId, idx);
+ addIndex(indexDefinition, tableId,
indexDefinition.name().hashCode(), idx);
}));
assertThat(createIndexFuture, willCompleteSuccessfully());
diff --git
a/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/AbstractPageMemoryTableStorage.java
b/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/AbstractPageMemoryTableStorage.java
index 6d9514975b..3379fdad7a 100644
---
a/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/AbstractPageMemoryTableStorage.java
+++
b/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/AbstractPageMemoryTableStorage.java
@@ -216,7 +216,7 @@ public abstract class AbstractPageMemoryTableStorage
implements MvTableStorage {
}
@Override
- public CompletableFuture<Void> destroyIndex(UUID indexId) {
+ public CompletableFuture<Void> destroyIndex(int indexId) {
throw new UnsupportedOperationException("Not implemented yet");
}
diff --git
a/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/index/AbstractPageMemoryIndexStorage.java
b/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/index/AbstractPageMemoryIndexStorage.java
index 2d0c315b72..cb7c502935 100644
---
a/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/index/AbstractPageMemoryIndexStorage.java
+++
b/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/index/AbstractPageMemoryIndexStorage.java
@@ -45,7 +45,7 @@ import org.jetbrains.annotations.Nullable;
*/
public abstract class AbstractPageMemoryIndexStorage implements IndexStorage {
/** Index ID. */
- private final UUID indexId;
+ private final int indexId;
/** Partition id. */
protected final int partitionId;
diff --git
a/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/index/meta/IndexMeta.java
b/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/index/meta/IndexMeta.java
index def8834b7f..afb928b440 100644
---
a/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/index/meta/IndexMeta.java
+++
b/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/index/meta/IndexMeta.java
@@ -40,7 +40,7 @@ public class IndexMeta extends IndexMetaKey {
* @param nextRowIdUuidToBuild Row ID uuid for which the index needs to be
built, {@code null} means that the index building has
* completed.
*/
- public IndexMeta(UUID id, long metaPageId, @Nullable UUID
nextRowIdUuidToBuild) {
+ public IndexMeta(int id, long metaPageId, @Nullable UUID
nextRowIdUuidToBuild) {
super(id);
this.metaPageId = metaPageId;
diff --git
a/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/index/meta/IndexMetaKey.java
b/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/index/meta/IndexMetaKey.java
index 90d55abd89..86050bbaab 100644
---
a/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/index/meta/IndexMetaKey.java
+++
b/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/index/meta/IndexMetaKey.java
@@ -17,27 +17,25 @@
package org.apache.ignite.internal.storage.pagememory.index.meta;
-import java.util.UUID;
-
/**
* Index meta key.
*/
public class IndexMetaKey {
- private final UUID indexId;
+ private final int indexId;
/**
* Constructor.
*
* @param indexId Index ID.
*/
- public IndexMetaKey(UUID indexId) {
+ public IndexMetaKey(int indexId) {
this.indexId = indexId;
}
/**
* Returns the index ID.
*/
- public UUID indexId() {
+ public int indexId() {
return indexId;
}
}
diff --git
a/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/index/meta/io/IndexMetaIo.java
b/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/index/meta/io/IndexMetaIo.java
index 29412de551..c299e089a6 100644
---
a/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/index/meta/io/IndexMetaIo.java
+++
b/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/index/meta/io/IndexMetaIo.java
@@ -17,7 +17,9 @@
package org.apache.ignite.internal.storage.pagememory.index.meta.io;
+import static org.apache.ignite.internal.pagememory.util.PageUtils.getInt;
import static org.apache.ignite.internal.pagememory.util.PageUtils.getLong;
+import static org.apache.ignite.internal.pagememory.util.PageUtils.putInt;
import static org.apache.ignite.internal.pagememory.util.PageUtils.putLong;
import java.util.UUID;
@@ -31,20 +33,17 @@ import
org.apache.ignite.internal.storage.pagememory.index.meta.IndexMetaKey;
*
* <p>Defines a following data layout:
* <ul>
- * <li>Index ID - {@link UUID} (16 bytes);</li>
+ * <li>Index ID - int (4 bytes);</li>
* <li>Index root page ID - long (8 bytes);</li>
* <li>Row ID uuid for which the index needs to be built - {@link UUID}
(16 bytes).</li>
* </ul>
*/
public interface IndexMetaIo {
- /** Offset of the {@link UUID#getMostSignificantBits() most significant
bits} of the index ID (8 bytes). */
- int INDEX_ID_MSB_OFFSET = 0;
-
- /** Offset of the {@link UUID#getLeastSignificantBits() least significant
bits} of the index ID (8 bytes). */
- int INDEX_ID_LSB_OFFSET = INDEX_ID_MSB_OFFSET + Long.BYTES;
+ /** Offset of the index ID (4 bytes). */
+ int INDEX_ID_OFFSET = 0;
/** Index tree meta page id offset - long (8 bytes). */
- int INDEX_TREE_META_PAGE_ID_OFFSET = INDEX_ID_LSB_OFFSET + Long.BYTES;
+ int INDEX_TREE_META_PAGE_ID_OFFSET = INDEX_ID_OFFSET + Integer.BYTES;
/**
* Offset of the {@link UUID#getMostSignificantBits() most significant
bits} of Row ID uuid for which the index needs to be built (8
@@ -59,7 +58,7 @@ public interface IndexMetaIo {
int NEXT_ROW_ID_TO_BUILT_LSB_OFFSET = NEXT_ROW_ID_TO_BUILT_MSB_OFFSET +
Long.BYTES;
/** Payload size in bytes. */
- int SIZE_IN_BYTES = 2 * Long.BYTES /* Index ID - {@link UUID} (16 bytes) */
+ int SIZE_IN_BYTES = Integer.BYTES /* Index ID - int (4 bytes) */
+ Long.BYTES /* Index root page ID - long (8 bytes) */
+ 2 * Long.BYTES /* Row ID uuid for which the index needs to be
built - {@link UUID} (16 bytes) */;
@@ -81,13 +80,10 @@ public interface IndexMetaIo {
default int compare(long pageAddr, int idx, IndexMetaKey indexMeta) {
int elementOffset = offset(idx);
- int cmp = Long.compare(getLong(pageAddr, elementOffset +
INDEX_ID_MSB_OFFSET), indexMeta.indexId().getMostSignificantBits());
-
- if (cmp != 0) {
- return cmp;
- }
-
- return Long.compare(getLong(pageAddr, elementOffset +
INDEX_ID_LSB_OFFSET), indexMeta.indexId().getLeastSignificantBits());
+ return Integer.compare(
+ getInt(pageAddr, elementOffset + INDEX_ID_OFFSET),
+ indexMeta.indexId()
+ );
}
/**
@@ -99,8 +95,7 @@ public interface IndexMetaIo {
default IndexMeta getRow(long pageAddr, int idx) {
int elementOffset = offset(idx);
- long indexIdMsb = getLong(pageAddr, elementOffset +
INDEX_ID_MSB_OFFSET);
- long indexIdLsb = getLong(pageAddr, elementOffset +
INDEX_ID_LSB_OFFSET);
+ int indexId = getInt(pageAddr, elementOffset + INDEX_ID_OFFSET);
long indexTreeMetaPageId = getLong(pageAddr, elementOffset +
INDEX_TREE_META_PAGE_ID_OFFSET);
@@ -111,7 +106,7 @@ public interface IndexMetaIo {
? null
: new UUID(nextRowIdUuidToBuiltMsb, nextRowIdUuidToBuiltLsb);
- return new IndexMeta(new UUID(indexIdMsb, indexIdLsb),
indexTreeMetaPageId, nextRowIdUuid);
+ return new IndexMeta(indexId, indexTreeMetaPageId, nextRowIdUuid);
}
/**
@@ -136,8 +131,7 @@ public interface IndexMetaIo {
IndexMeta row = (IndexMeta) rowKey;
- putLong(pageAddr, off + INDEX_ID_MSB_OFFSET,
row.indexId().getMostSignificantBits());
- putLong(pageAddr, off + INDEX_ID_LSB_OFFSET,
row.indexId().getLeastSignificantBits());
+ putInt(pageAddr, off + INDEX_ID_OFFSET, row.indexId());
putLong(pageAddr, off + INDEX_TREE_META_PAGE_ID_OFFSET,
row.metaPageId());
diff --git
a/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/mv/AbstractPageMemoryMvPartitionStorage.java
b/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/mv/AbstractPageMemoryMvPartitionStorage.java
index c1dd67cb79..9a06d9c68e 100644
---
a/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/mv/AbstractPageMemoryMvPartitionStorage.java
+++
b/modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/mv/AbstractPageMemoryMvPartitionStorage.java
@@ -121,9 +121,9 @@ public abstract class AbstractPageMemoryMvPartitionStorage
implements MvPartitio
protected final DataPageReader rowVersionDataPageReader;
- protected final ConcurrentMap<UUID, PageMemoryHashIndexStorage>
hashIndexes = new ConcurrentHashMap<>();
+ protected final ConcurrentMap<Integer, PageMemoryHashIndexStorage>
hashIndexes = new ConcurrentHashMap<>();
- protected final ConcurrentMap<UUID, PageMemorySortedIndexStorage>
sortedIndexes = new ConcurrentHashMap<>();
+ protected final ConcurrentMap<Integer, PageMemorySortedIndexStorage>
sortedIndexes = new ConcurrentHashMap<>();
/** Busy lock. */
protected final IgniteSpinBusyLock busyLock = new IgniteSpinBusyLock();
@@ -183,9 +183,12 @@ public abstract class AbstractPageMemoryMvPartitionStorage
implements MvPartitio
TablesView tablesView =
tableStorage.tablesConfiguration().value();
for (IndexMeta indexMeta : cursor) {
- UUID indexId = indexMeta.indexId();
+ int indexId = indexMeta.indexId();
- TableIndexView indexCfgView =
tablesView.indexes().get(indexId);
+ TableIndexView indexCfgView = tablesView.indexes().stream()
+ .filter(tableIndexView -> indexId ==
tableIndexView.id())
+ .findFirst()
+ .orElse(null);
if (indexCfgView instanceof HashIndexView) {
hashIndexes.put(
@@ -984,7 +987,7 @@ public abstract class AbstractPageMemoryMvPartitionStorage
implements MvPartitio
return removeWriteOnGc.getResult();
}
- IndexMeta createIndexMetaForNewIndex(UUID indexId) {
+ IndexMeta createIndexMetaForNewIndex(int indexId) {
return new IndexMeta(indexId, 0L,
RowId.lowestRowId(partitionId).uuid());
}
diff --git
a/modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/ColumnFamilyUtils.java
b/modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/ColumnFamilyUtils.java
index 8eb1ab722b..1053ce846d 100644
---
a/modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/ColumnFamilyUtils.java
+++
b/modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/ColumnFamilyUtils.java
@@ -18,7 +18,6 @@
package org.apache.ignite.internal.storage.rocksdb;
import java.nio.charset.StandardCharsets;
-import java.util.UUID;
import org.rocksdb.RocksDB;
/**
@@ -71,11 +70,10 @@ class ColumnFamilyUtils {
* Creates a column family name by index ID.
*
* @param indexId Index ID.
- * @return Column family name.
*
* @see #sortedIndexId
*/
- static String sortedIndexCfName(UUID indexId) {
+ static String sortedIndexCfName(int indexId) {
return SORTED_INDEX_CF_PREFIX + indexId;
}
@@ -83,11 +81,10 @@ class ColumnFamilyUtils {
* Extracts a Sorted Index ID from the given Column Family name.
*
* @param cfName Column Family name.
- * @return Sorted Index ID.
*
* @see #sortedIndexCfName
*/
- static UUID sortedIndexId(String cfName) {
- return
UUID.fromString(cfName.substring(SORTED_INDEX_CF_PREFIX.length()));
+ static int sortedIndexId(String cfName) {
+ return
Integer.parseInt(cfName.substring(SORTED_INDEX_CF_PREFIX.length()));
}
}
diff --git
a/modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/RocksDbMetaStorage.java
b/modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/RocksDbMetaStorage.java
index 32cb978b64..3b7eeffadd 100644
---
a/modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/RocksDbMetaStorage.java
+++
b/modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/RocksDbMetaStorage.java
@@ -27,7 +27,6 @@ import static
org.apache.ignite.internal.storage.rocksdb.RocksDbStorageUtils.put
import static org.apache.ignite.internal.util.ArrayUtils.BYTE_EMPTY_ARRAY;
import java.nio.ByteBuffer;
-import java.util.UUID;
import java.util.stream.Stream;
import org.apache.ignite.internal.rocksdb.ColumnFamily;
import org.apache.ignite.internal.rocksdb.RocksUtils;
@@ -131,7 +130,7 @@ public class RocksDbMetaStorage {
* @param partitionId Partition ID.
* @param ifAbsent Will be returned if next the row ID for which the index
needs to be built has never been saved.
*/
- public @Nullable RowId getNextRowIdToBuilt(UUID indexId, int partitionId,
RowId ifAbsent) {
+ public @Nullable RowId getNextRowIdToBuilt(int indexId, int partitionId,
RowId ifAbsent) {
try {
byte[] lastBuiltRowIdBytes =
metaColumnFamily.get(indexMetaKey(partitionId, indexId));
@@ -161,7 +160,7 @@ public class RocksDbMetaStorage {
* @param indexId Index ID.
* @param rowId Row ID.
*/
- public void putNextRowIdToBuilt(AbstractWriteBatch writeBatch, UUID
indexId, int partitionId, @Nullable RowId rowId) {
+ public void putNextRowIdToBuilt(AbstractWriteBatch writeBatch, int
indexId, int partitionId, @Nullable RowId rowId) {
try {
writeBatch.put(metaColumnFamily.handle(),
indexMetaKey(partitionId, indexId), indexLastBuildRowId(rowId));
} catch (RocksDBException e) {
@@ -173,7 +172,7 @@ public class RocksDbMetaStorage {
}
}
- private static byte[] indexMetaKey(int partitionId, UUID indexId) {
+ private static byte[] indexMetaKey(int partitionId, int indexId) {
assert partitionId >= 0 && partitionId <= 0xFFFF : partitionId;
ByteBuffer buffer =
ByteBuffer.allocate(INDEX_META_KEY_SIZE).order(KEY_BYTE_ORDER);
diff --git
a/modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/RocksDbStorageUtils.java
b/modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/RocksDbStorageUtils.java
index 309ff84709..6cae5459fc 100644
---
a/modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/RocksDbStorageUtils.java
+++
b/modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/RocksDbStorageUtils.java
@@ -36,7 +36,7 @@ public class RocksDbStorageUtils {
public static final int PARTITION_ID_SIZE = Short.BYTES;
/** Index ID size in bytes. */
- public static final int INDEX_ID_SIZE = 2 * Long.BYTES;
+ public static final int INDEX_ID_SIZE = Integer.SIZE;
static void putRowIdUuid(ByteBuffer keyBuffer, UUID rowIdUuid) {
assert keyBuffer.order() == KEY_BYTE_ORDER;
@@ -45,11 +45,10 @@ public class RocksDbStorageUtils {
keyBuffer.putLong(normalize(rowIdUuid.getLeastSignificantBits()));
}
- static void putIndexId(ByteBuffer keyBuffer, UUID indexId) {
+ static void putIndexId(ByteBuffer keyBuffer, int indexId) {
assert keyBuffer.order() == KEY_BYTE_ORDER;
- keyBuffer.putLong(normalize(indexId.getMostSignificantBits()));
- keyBuffer.putLong(normalize(indexId.getLeastSignificantBits()));
+ keyBuffer.putInt(indexId);
}
static UUID getRowIdUuid(ByteBuffer keyBuffer, int offset) {
diff --git
a/modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/RocksDbTableStorage.java
b/modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/RocksDbTableStorage.java
index c6cec576cb..f75c329215 100644
---
a/modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/RocksDbTableStorage.java
+++
b/modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/RocksDbTableStorage.java
@@ -133,10 +133,10 @@ public class RocksDbTableStorage implements
MvTableStorage {
private volatile MvPartitionStorages<RocksDbMvPartitionStorage>
mvPartitionStorages;
/** Hash Index storages by Index IDs. */
- private final ConcurrentMap<UUID, HashIndex> hashIndices = new
ConcurrentHashMap<>();
+ private final ConcurrentMap<Integer, HashIndex> hashIndices = new
ConcurrentHashMap<>();
/** Sorted Index storages by Index IDs. */
- private final ConcurrentMap<UUID, SortedIndex> sortedIndices = new
ConcurrentHashMap<>();
+ private final ConcurrentMap<Integer, SortedIndex> sortedIndices = new
ConcurrentHashMap<>();
/** Busy lock to stop synchronously. */
private final IgniteSpinBusyLock busyLock = new IgniteSpinBusyLock();
@@ -250,7 +250,7 @@ public class RocksDbTableStorage implements MvTableStorage {
try {
db = RocksDB.open(dbOptions,
tablePath.toAbsolutePath().toString(), cfDescriptors, cfHandles);
- Map<UUID, ColumnFamily> sortedIndexColumnFamilyByIndexId = new
HashMap<>();
+ Map<Integer, ColumnFamily> sortedIndexColumnFamilyByIndexId =
new HashMap<>();
// read all existing Column Families from the db and parse
them according to type: meta, partition data or index.
for (ColumnFamilyHandle cfHandle : cfHandles) {
@@ -291,8 +291,8 @@ public class RocksDbTableStorage implements MvTableStorage {
assert partitionCf != null;
assert hashIndexCf != null;
- for (Entry<UUID, ColumnFamily> entry :
sortedIndexColumnFamilyByIndexId.entrySet()) {
- UUID indexId = entry.getKey();
+ for (Entry<Integer, ColumnFamily> entry :
sortedIndexColumnFamilyByIndexId.entrySet()) {
+ int indexId = entry.getKey();
var indexDescriptor = new SortedIndexDescriptor(indexId,
tablesCfg.value());
@@ -515,7 +515,7 @@ public class RocksDbTableStorage implements MvTableStorage {
}
@Override
- public CompletableFuture<Void> destroyIndex(UUID indexId) {
+ public CompletableFuture<Void> destroyIndex(int indexId) {
return inBusyLock(busyLock, () -> {
HashIndex hashIdx = hashIndices.remove(indexId);
diff --git
a/modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/index/AbstractRocksDbIndexStorage.java
b/modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/index/AbstractRocksDbIndexStorage.java
index 0984c6eade..f51148e7ce 100644
---
a/modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/index/AbstractRocksDbIndexStorage.java
+++
b/modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/index/AbstractRocksDbIndexStorage.java
@@ -21,7 +21,6 @@ import static
org.apache.ignite.internal.storage.util.StorageUtils.throwExceptio
import static
org.apache.ignite.internal.storage.util.StorageUtils.throwExceptionDependingOnStorageStateOnRebalance;
import static
org.apache.ignite.internal.storage.util.StorageUtils.throwExceptionIfStorageInProgressOfRebalance;
-import java.util.UUID;
import java.util.concurrent.atomic.AtomicReference;
import java.util.function.Supplier;
import org.apache.ignite.internal.storage.RowId;
@@ -41,7 +40,7 @@ import org.rocksdb.WriteBatchWithIndex;
* Abstract index storage base on RocksDB.
*/
abstract class AbstractRocksDbIndexStorage implements IndexStorage {
- private final UUID indexId;
+ private final int indexId;
final PartitionDataHelper helper;
@@ -56,7 +55,7 @@ abstract class AbstractRocksDbIndexStorage implements
IndexStorage {
/** Row ID for which the index needs to be built, {@code null} means that
the index building has completed. */
private volatile @Nullable RowId nextRowIdToBuilt;
- AbstractRocksDbIndexStorage(UUID indexId, PartitionDataHelper helper,
RocksDbMetaStorage indexMetaStorage) {
+ AbstractRocksDbIndexStorage(int indexId, PartitionDataHelper helper,
RocksDbMetaStorage indexMetaStorage) {
this.indexId = indexId;
this.helper = helper;
this.indexMetaStorage = indexMetaStorage;
diff --git
a/modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/index/RocksDbHashIndexStorage.java
b/modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/index/RocksDbHashIndexStorage.java
index a9159fc4a5..8324f1779a 100644
---
a/modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/index/RocksDbHashIndexStorage.java
+++
b/modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/index/RocksDbHashIndexStorage.java
@@ -27,7 +27,6 @@ import static
org.apache.ignite.internal.util.ArrayUtils.BYTE_EMPTY_ARRAY;
import static org.apache.ignite.internal.util.ByteUtils.bytesToLong;
import java.nio.ByteBuffer;
-import java.util.UUID;
import org.apache.ignite.internal.rocksdb.ColumnFamily;
import org.apache.ignite.internal.rocksdb.RocksIteratorAdapter;
import org.apache.ignite.internal.rocksdb.RocksUtils;
@@ -93,12 +92,9 @@ public class RocksDbHashIndexStorage extends
AbstractRocksDbIndexStorage impleme
this.descriptor = descriptor;
this.indexCf = indexCf;
- UUID indexId = descriptor.id();
-
this.constantPrefix = ByteBuffer.allocate(INDEX_ID_SIZE +
PARTITION_ID_SIZE)
.order(KEY_BYTE_ORDER)
- .putLong(indexId.getMostSignificantBits())
- .putLong(indexId.getLeastSignificantBits())
+ .putInt(descriptor.id())
.putShort((short) helper.partitionId())
.array();
}
diff --git
a/modules/table/src/integrationTest/java/org/apache/ignite/distributed/ItTxDistributedTestSingleNode.java
b/modules/table/src/integrationTest/java/org/apache/ignite/distributed/ItTxDistributedTestSingleNode.java
index 0712a827d3..77a39afd84 100644
---
a/modules/table/src/integrationTest/java/org/apache/ignite/distributed/ItTxDistributedTestSingleNode.java
+++
b/modules/table/src/integrationTest/java/org/apache/ignite/distributed/ItTxDistributedTestSingleNode.java
@@ -37,7 +37,6 @@ import java.util.List;
import java.util.Map;
import java.util.Map.Entry;
import java.util.Set;
-import java.util.UUID;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.CopyOnWriteArrayList;
import java.util.concurrent.ScheduledThreadPoolExecutor;
@@ -429,6 +428,8 @@ public class ItTxDistributedTestSingleNode extends
TxAbstractTest {
List<CompletableFuture<Void>> partitionReadyFutures = new
ArrayList<>();
+ int globalIndexId = 1;
+
for (int p = 0; p < assignments.size(); p++) {
Set<String> partAssignments = assignments.get(p);
@@ -446,7 +447,7 @@ public class ItTxDistributedTestSingleNode extends
TxAbstractTest {
int partId = p;
- UUID indexId = UUID.randomUUID();
+ int indexId = globalIndexId++;
Function<BinaryRow, BinaryTuple> row2Tuple =
BinaryRowConverter.keyExtractor(schemaDescriptor);
diff --git
a/modules/table/src/main/java/org/apache/ignite/internal/table/InternalTable.java
b/modules/table/src/main/java/org/apache/ignite/internal/table/InternalTable.java
index 86c2676afe..6946702e79 100644
---
a/modules/table/src/main/java/org/apache/ignite/internal/table/InternalTable.java
+++
b/modules/table/src/main/java/org/apache/ignite/internal/table/InternalTable.java
@@ -38,7 +38,6 @@ import
org.apache.ignite.internal.util.PendingComparableValuesTracker;
import org.apache.ignite.internal.utils.PrimaryReplica;
import org.apache.ignite.network.ClusterNode;
import org.apache.ignite.tx.TransactionException;
-import org.jetbrains.annotations.NotNull;
import org.jetbrains.annotations.Nullable;
/**
@@ -56,7 +55,7 @@ public interface InternalTable extends ManuallyCloseable {
/**
* Gets a table id.
*
- * @return Table id as UUID.
+ * @return Table id.
*/
int tableId();
@@ -96,8 +95,8 @@ public interface InternalTable extends ManuallyCloseable {
*/
CompletableFuture<BinaryRow> get(
BinaryRowEx keyRow,
- @NotNull HybridTimestamp readTimestamp,
- @NotNull ClusterNode recipientNode
+ HybridTimestamp readTimestamp,
+ ClusterNode recipientNode
);
/**
@@ -119,8 +118,8 @@ public interface InternalTable extends ManuallyCloseable {
*/
CompletableFuture<Collection<BinaryRow>> getAll(
Collection<BinaryRowEx> keyRows,
- @NotNull HybridTimestamp readTimestamp,
- @NotNull ClusterNode recipientNode
+ HybridTimestamp readTimestamp,
+ ClusterNode recipientNode
);
@@ -275,8 +274,8 @@ public interface InternalTable extends ManuallyCloseable {
*/
default Publisher<BinaryRow> scan(
int partId,
- @NotNull HybridTimestamp readTimestamp,
- @NotNull ClusterNode recipientNode
+ HybridTimestamp readTimestamp,
+ ClusterNode recipientNode
) {
return scan(partId, readTimestamp, recipientNode, null, null, null, 0,
null);
}
@@ -297,9 +296,9 @@ public interface InternalTable extends ManuallyCloseable {
*/
Publisher<BinaryRow> scan(
int partId,
- @NotNull HybridTimestamp readTimestamp,
- @NotNull ClusterNode recipientNode,
- @NotNull UUID indexId,
+ HybridTimestamp readTimestamp,
+ ClusterNode recipientNode,
+ @Nullable Integer indexId,
@Nullable BinaryTuplePrefix lowerBound,
@Nullable BinaryTuplePrefix upperBound,
int flags,
@@ -322,7 +321,7 @@ public interface InternalTable extends ManuallyCloseable {
int partId,
UUID txId,
PrimaryReplica recipient,
- @Nullable UUID indexId,
+ @Nullable Integer indexId,
@Nullable BinaryTuplePrefix lowerBound,
@Nullable BinaryTuplePrefix upperBound,
int flags,
@@ -344,7 +343,7 @@ public interface InternalTable extends ManuallyCloseable {
Publisher<BinaryRow> scan(
int partId,
@Nullable InternalTransaction tx,
- @Nullable UUID indexId,
+ @Nullable Integer indexId,
@Nullable BinaryTuplePrefix lowerBound,
@Nullable BinaryTuplePrefix upperBound,
int flags,
@@ -364,9 +363,9 @@ public interface InternalTable extends ManuallyCloseable {
*/
Publisher<BinaryRow> lookup(
int partId,
- @NotNull HybridTimestamp readTimestamp,
- @NotNull ClusterNode recipientNode,
- @NotNull UUID indexId,
+ HybridTimestamp readTimestamp,
+ ClusterNode recipientNode,
+ int indexId,
BinaryTuple key,
@Nullable BitSet columnsToInclude
);
@@ -387,7 +386,7 @@ public interface InternalTable extends ManuallyCloseable {
int partId,
UUID txId,
PrimaryReplica recipient,
- UUID indexId,
+ int indexId,
BinaryTuple key,
@Nullable BitSet columnsToInclude
);
diff --git
a/modules/table/src/main/java/org/apache/ignite/internal/table/TableImpl.java
b/modules/table/src/main/java/org/apache/ignite/internal/table/TableImpl.java
index 36b8933810..f4fe7ead47 100644
---
a/modules/table/src/main/java/org/apache/ignite/internal/table/TableImpl.java
+++
b/modules/table/src/main/java/org/apache/ignite/internal/table/TableImpl.java
@@ -20,12 +20,10 @@ package org.apache.ignite.internal.table;
import static java.util.concurrent.CompletableFuture.allOf;
import java.util.ArrayList;
-import java.util.Collection;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Objects;
-import java.util.UUID;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.ConcurrentHashMap;
import java.util.function.Function;
@@ -69,12 +67,12 @@ public class TableImpl implements Table {
/** Schema registry. Should be set either in constructor or via {@link
#schemaView(SchemaRegistry)} before start of using the table. */
private volatile SchemaRegistry schemaReg;
- private final CompletableFuture<UUID> pkId = new CompletableFuture<>();
+ private final CompletableFuture<Integer> pkId = new CompletableFuture<>();
- private final Map<UUID, CompletableFuture<?>> indexesToWait = new
ConcurrentHashMap<>();
+ private final Map<Integer, CompletableFuture<?>> indexesToWait = new
ConcurrentHashMap<>();
- private final Map<UUID, IndexStorageAdapterFactory>
indexStorageAdapterFactories = new ConcurrentHashMap<>();
- private final Map<UUID, IndexLockerFactory> indexLockerFactories = new
ConcurrentHashMap<>();
+ private final Map<Integer, IndexStorageAdapterFactory>
indexStorageAdapterFactories = new ConcurrentHashMap<>();
+ private final Map<Integer, IndexLockerFactory> indexLockerFactories = new
ConcurrentHashMap<>();
/**
* Constructor.
@@ -115,12 +113,12 @@ public class TableImpl implements Table {
*
* @param pkId An identifier of a primary index.
*/
- public void pkId(UUID pkId) {
- this.pkId.complete(Objects.requireNonNull(pkId, "pkId"));
+ public void pkId(int pkId) {
+ this.pkId.complete(pkId);
}
/** Returns an identifier of a primary index. */
- public UUID pkId() {
+ public int pkId() {
return pkId.join();
}
@@ -228,12 +226,12 @@ public class TableImpl implements Table {
public TableIndexStoragesSupplier indexStorageAdapters(int partId) {
return new TableIndexStoragesSupplier() {
@Override
- public Map<UUID, TableSchemaAwareIndexStorage> get() {
+ public Map<Integer, TableSchemaAwareIndexStorage> get() {
awaitIndexes();
List<IndexStorageAdapterFactory> factories = new
ArrayList<>(indexStorageAdapterFactories.values());
- Map<UUID, TableSchemaAwareIndexStorage> adapters = new
HashMap<>();
+ Map<Integer, TableSchemaAwareIndexStorage> adapters = new
HashMap<>();
for (IndexStorageAdapterFactory factory : factories) {
TableSchemaAwareIndexStorage storage =
factory.create(partId);
@@ -244,20 +242,20 @@ public class TableImpl implements Table {
}
@Override
- public void addIndexToWaitIfAbsent(UUID indexId) {
- addIndexesToWait(List.of(indexId));
+ public void addIndexToWaitIfAbsent(int indexId) {
+ addIndexesToWait(indexId);
}
};
}
/** Returns a supplier of index locker factories for given partition. */
- public Supplier<Map<UUID, IndexLocker>> indexesLockers(int partId) {
+ public Supplier<Map<Integer, IndexLocker>> indexesLockers(int partId) {
return () -> {
awaitIndexes();
List<IndexLockerFactory> factories = new
ArrayList<>(indexLockerFactories.values());
- Map<UUID, IndexLocker> lockers = new HashMap<>(factories.size());
+ Map<Integer, IndexLocker> lockers = new
HashMap<>(factories.size());
for (IndexLockerFactory factory : factories) {
IndexLocker locker = factory.create(partId);
@@ -293,7 +291,7 @@ public class TableImpl implements Table {
boolean unique,
Function<BinaryRow, BinaryTuple> searchRowResolver
) {
- UUID indexId = indexDescriptor.id();
+ int indexId = indexDescriptor.id();
indexLockerFactories.put(
indexId,
@@ -326,7 +324,7 @@ public class TableImpl implements Table {
SortedIndexDescriptor indexDescriptor,
Function<BinaryRow, BinaryTuple> searchRowResolver
) {
- UUID indexId = indexDescriptor.id();
+ int indexId = indexDescriptor.id();
indexLockerFactories.put(
indexId,
@@ -355,7 +353,7 @@ public class TableImpl implements Table {
*
* @param indexId An index id to unregister.
*/
- public void unregisterIndex(UUID indexId) {
+ public void unregisterIndex(int indexId) {
indexLockerFactories.remove(indexId);
indexStorageAdapterFactories.remove(indexId);
@@ -406,8 +404,8 @@ public class TableImpl implements Table {
* @param indexIds Indexes Index IDs.
*/
// TODO: IGNITE-19082 Needs to be redone/improved
- public void addIndexesToWait(Collection<UUID> indexIds) {
- for (UUID indexId : indexIds) {
+ public void addIndexesToWait(int... indexIds) {
+ for (int indexId : indexIds) {
indexesToWait.compute(indexId, (indexId0, awaitIndexFuture) -> {
if (awaitIndexFuture != null) {
return awaitIndexFuture;
@@ -423,7 +421,7 @@ public class TableImpl implements Table {
}
}
- private void completeWaitIndex(UUID indexId) {
+ private void completeWaitIndex(int indexId) {
CompletableFuture<?> indexToWaitFuture = indexesToWait.remove(indexId);
if (indexToWaitFuture != null) {
diff --git
a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/HashIndexLocker.java
b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/HashIndexLocker.java
index 5cd9e5b9b3..c28a0bfca2 100644
---
a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/HashIndexLocker.java
+++
b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/HashIndexLocker.java
@@ -34,7 +34,7 @@ import org.apache.ignite.internal.tx.LockMode;
* <p>Simply acquires lock on a given row.
*/
public class HashIndexLocker implements IndexLocker {
- private final UUID indexId;
+ private final int indexId;
private final LockMode modificationMode;
private final LockManager lockManager;
private final Function<BinaryRow, BinaryTuple> indexRowResolver;
@@ -46,7 +46,7 @@ public class HashIndexLocker implements IndexLocker {
* @param lockManager A lock manager to acquire locks in.
* @param indexRowResolver A convertor which derives an index key from
given table row.
*/
- public HashIndexLocker(UUID indexId, boolean unique, LockManager
lockManager,
+ public HashIndexLocker(int indexId, boolean unique, LockManager
lockManager,
Function<BinaryRow, BinaryTuple> indexRowResolver) {
this.indexId = indexId;
this.modificationMode = unique ? LockMode.X : LockMode.IX;
@@ -54,9 +54,8 @@ public class HashIndexLocker implements IndexLocker {
this.indexRowResolver = indexRowResolver;
}
- /** {@inheritDoc} */
@Override
- public UUID id() {
+ public int id() {
return indexId;
}
diff --git
a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/IndexLocker.java
b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/IndexLocker.java
index 57e2915587..4f6f8d43ee 100644
---
a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/IndexLocker.java
+++
b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/IndexLocker.java
@@ -30,7 +30,7 @@ import org.apache.ignite.internal.tx.Lock;
*/
public interface IndexLocker {
/** Returns an identifier of the index this locker created for. */
- UUID id();
+ int id();
/**
* Acquires the lock for lookup operation.
diff --git
a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/SortedIndexLocker.java
b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/SortedIndexLocker.java
index c70933aed3..9720cc14f5 100644
---
a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/SortedIndexLocker.java
+++
b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/SortedIndexLocker.java
@@ -43,7 +43,7 @@ public class SortedIndexLocker implements IndexLocker {
/** Index INF+ value object. */
private final Object positiveInf;
- private final UUID indexId;
+ private final int indexId;
private final LockManager lockManager;
@@ -61,7 +61,7 @@ public class SortedIndexLocker implements IndexLocker {
* @param storage A storage of an index this locker is created for.
* @param indexRowResolver A convertor which derives an index key from
given table row.
*/
- public SortedIndexLocker(UUID indexId, int partId, LockManager
lockManager, SortedIndexStorage storage,
+ public SortedIndexLocker(int indexId, int partId, LockManager lockManager,
SortedIndexStorage storage,
Function<BinaryRow, BinaryTuple> indexRowResolver) {
this.indexId = indexId;
this.lockManager = lockManager;
@@ -70,9 +70,8 @@ public class SortedIndexLocker implements IndexLocker {
this.positiveInf = Integer.valueOf(partId);
}
- /** {@inheritDoc} */
@Override
- public UUID id() {
+ public int id() {
return indexId;
}
diff --git
a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/TableIndexStoragesSupplier.java
b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/TableIndexStoragesSupplier.java
index bd0bb96b49..2398aed61b 100644
---
a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/TableIndexStoragesSupplier.java
+++
b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/TableIndexStoragesSupplier.java
@@ -18,7 +18,6 @@
package org.apache.ignite.internal.table.distributed;
import java.util.Map;
-import java.util.UUID;
/**
* Supplier table index storages.
@@ -29,12 +28,12 @@ public interface TableIndexStoragesSupplier {
*
* <p>Waits for the primary key index and all other registered indexes to
be created.
*/
- Map<UUID, TableSchemaAwareIndexStorage> get();
+ Map<Integer, TableSchemaAwareIndexStorage> get();
/**
* Adds index creation waits if it hasn't been created yet.
*
* @param indexId Index ID.
*/
- void addIndexToWaitIfAbsent(UUID indexId);
+ void addIndexToWaitIfAbsent(int indexId);
}
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 310c12c14d..1e1b6c0aa6 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
@@ -74,7 +74,6 @@ import java.util.stream.Stream;
import org.apache.ignite.configuration.ConfigurationChangeException;
import org.apache.ignite.configuration.ConfigurationProperty;
import org.apache.ignite.configuration.NamedConfigurationTree;
-import org.apache.ignite.configuration.NamedListView;
import
org.apache.ignite.configuration.notifications.ConfigurationNamedListListener;
import
org.apache.ignite.configuration.notifications.ConfigurationNotificationEvent;
import org.apache.ignite.internal.affinity.AffinityUtils;
@@ -1111,7 +1110,7 @@ public class TableManager extends Producer<TableEvent,
TableEventParameters> imp
var table = new TableImpl(internalTable, lockMgr);
// TODO: IGNITE-19082 Need another way to wait for indexes
- table.addIndexesToWait(collectTableIndexes(tblId));
+ table.addIndexesToWait(collectTableIndexIds(tblId));
tablesByIdVv.update(causalityToken, (previous, e) ->
inBusyLock(busyLock, () -> {
if (e != null) {
@@ -2384,20 +2383,11 @@ public class TableManager extends Producer<TableEvent,
TableEventParameters> imp
}
}
- private Collection<UUID> collectTableIndexes(int tableId) {
- NamedListView<? extends TableIndexView> indexes =
tablesCfg.value().indexes();
-
- List<UUID> indexIds = new ArrayList<>();
-
- for (int i = 0; i < indexes.size(); i++) {
- TableIndexView indexConfig = indexes.get(i);
-
- if (indexConfig.tableId() == tableId) {
- indexIds.add(indexConfig.id());
- }
- }
-
- return indexIds;
+ private int[] collectTableIndexIds(int tableId) {
+ return tablesCfg.value().indexes().stream()
+ .filter(tableIndexView -> tableIndexView.tableId() == tableId)
+ .mapToInt(TableIndexView::id)
+ .toArray();
}
private static void closePartitionTrackers(InternalTable internalTable,
int partitionId) {
diff --git
a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/TableSchemaAwareIndexStorage.java
b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/TableSchemaAwareIndexStorage.java
index b56f7c222d..854830ac94 100644
---
a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/TableSchemaAwareIndexStorage.java
+++
b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/TableSchemaAwareIndexStorage.java
@@ -17,7 +17,6 @@
package org.apache.ignite.internal.table.distributed;
-import java.util.UUID;
import java.util.function.Function;
import org.apache.ignite.internal.schema.BinaryRow;
import org.apache.ignite.internal.schema.BinaryTuple;
@@ -32,13 +31,13 @@ import org.apache.ignite.internal.util.Cursor;
* i.e. derives the index key from a given table row.
*/
public class TableSchemaAwareIndexStorage {
- private final UUID indexId;
+ private final int indexId;
private final IndexStorage storage;
private final Function<BinaryRow, BinaryTuple> indexRowResolver;
/** Constructs the object. */
public TableSchemaAwareIndexStorage(
- UUID indexId,
+ int indexId,
IndexStorage storage,
Function<BinaryRow, BinaryTuple> indexRowResolver
) {
@@ -48,7 +47,7 @@ public class TableSchemaAwareIndexStorage {
}
/** Returns an identifier of the index. */
- public UUID id() {
+ public int id() {
return indexId;
}
diff --git
a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/command/BuildIndexCommand.java
b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/command/BuildIndexCommand.java
index dd0527fa4a..8ebd616916 100644
---
a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/command/BuildIndexCommand.java
+++
b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/command/BuildIndexCommand.java
@@ -36,7 +36,7 @@ public interface BuildIndexCommand extends WriteCommand {
/**
* Returns index ID.
*/
- UUID indexId();
+ int indexId();
/**
* Returns row IDs for which to build indexes.
diff --git
a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/index/IndexBuildTaskId.java
b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/index/IndexBuildTaskId.java
index 333973c26c..71fbd71ae7 100644
---
a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/index/IndexBuildTaskId.java
+++
b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/index/IndexBuildTaskId.java
@@ -17,7 +17,6 @@
package org.apache.ignite.internal.table.distributed.index;
-import java.util.UUID;
import org.apache.ignite.internal.tostring.S;
/**
@@ -28,9 +27,9 @@ class IndexBuildTaskId {
private final int partitionId;
- private final UUID indexId;
+ private final int indexId;
- IndexBuildTaskId(int tableId, int partitionId, UUID indexId) {
+ IndexBuildTaskId(int tableId, int partitionId, int indexId) {
this.tableId = tableId;
this.partitionId = partitionId;
this.indexId = indexId;
@@ -44,7 +43,7 @@ class IndexBuildTaskId {
return partitionId;
}
- public UUID getIndexId() {
+ public int getIndexId() {
return indexId;
}
@@ -59,14 +58,14 @@ class IndexBuildTaskId {
IndexBuildTaskId that = (IndexBuildTaskId) o;
- return partitionId == that.partitionId && tableId == that.tableId &&
indexId.equals(that.indexId);
+ return partitionId == that.partitionId && tableId == that.tableId &&
indexId == that.indexId;
}
@Override
public int hashCode() {
int result = tableId;
result = 31 * result + partitionId;
- result = 31 * result + indexId.hashCode();
+ result = 31 * result + indexId;
return result;
}
diff --git
a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/index/IndexBuilder.java
b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/index/IndexBuilder.java
index 29c22897c5..150b7edf08 100644
---
a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/index/IndexBuilder.java
+++
b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/index/IndexBuilder.java
@@ -20,7 +20,6 @@ package org.apache.ignite.internal.table.distributed.index;
import java.util.Iterator;
import java.util.Map;
import java.util.Map.Entry;
-import java.util.UUID;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.LinkedBlockingQueue;
@@ -89,7 +88,7 @@ public class IndexBuilder implements ManuallyCloseable {
public void startBuildIndex(
int tableId,
int partitionId,
- UUID indexId,
+ int indexId,
IndexStorage indexStorage,
MvPartitionStorage partitionStorage,
RaftGroupService raftClient
@@ -123,7 +122,7 @@ public class IndexBuilder implements ManuallyCloseable {
* @param partitionId Partition ID.
* @param indexId Index ID.
*/
- public void stopBuildIndex(int tableId, int partitionId, UUID indexId) {
+ public void stopBuildIndex(int tableId, int partitionId, int indexId) {
inBusyLock(() -> {
IndexBuildTask removed = indexBuildTaskById.remove(new
IndexBuildTaskId(tableId, partitionId, indexId));
diff --git
a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/index/IndexUpdateHandler.java
b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/index/IndexUpdateHandler.java
index ea495fcf0d..778462c640 100644
---
a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/index/IndexUpdateHandler.java
+++
b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/index/IndexUpdateHandler.java
@@ -18,7 +18,6 @@
package org.apache.ignite.internal.table.distributed.index;
import java.nio.ByteBuffer;
-import java.util.UUID;
import java.util.stream.Stream;
import org.apache.ignite.internal.schema.BinaryRow;
import org.apache.ignite.internal.schema.BinaryTuple;
@@ -129,7 +128,7 @@ public class IndexUpdateHandler {
* @param rowStream Stream of rows to build the index without tombstones.
* @param nextRowIdToBuild Row ID for which the index needs to be build
next, {@code null} means that the index is build.
*/
- public void buildIndex(UUID indexId, Stream<BinaryRowAndRowId> rowStream,
@Nullable RowId nextRowIdToBuild) {
+ public void buildIndex(int indexId, Stream<BinaryRowAndRowId> rowStream,
@Nullable RowId nextRowIdToBuild) {
// TODO: IGNITE-19082 Need another way to wait for index creation
indexes.addIndexToWaitIfAbsent(indexId);
diff --git
a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/replication/request/ScanRetrieveBatchReplicaRequest.java
b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/replication/request/ScanRetrieveBatchReplicaRequest.java
index c785fb4429..cf67762f04 100644
---
a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/replication/request/ScanRetrieveBatchReplicaRequest.java
+++
b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/replication/request/ScanRetrieveBatchReplicaRequest.java
@@ -18,7 +18,6 @@
package org.apache.ignite.internal.table.distributed.replication.request;
import java.util.BitSet;
-import java.util.UUID;
import org.apache.ignite.internal.replicator.message.ReplicaRequest;
import org.apache.ignite.internal.schema.BinaryTuple;
import org.apache.ignite.internal.schema.BinaryTuplePrefix;
@@ -38,11 +37,8 @@ public interface ScanRetrieveBatchReplicaRequest extends
ReplicaRequest {
/**
* Gets an index to use fot the retrieve request.
- *
- * @return Index id.
*/
- @Nullable
- UUID indexToUse();
+ @Nullable Integer indexToUse();
/**
* Gets a key which is used for exact comparison in the index.
diff --git
a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/replicator/PartitionReplicaListener.java
b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/replicator/PartitionReplicaListener.java
index 03e4247077..c5c4b06369 100644
---
a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/replicator/PartitionReplicaListener.java
+++
b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/replicator/PartitionReplicaListener.java
@@ -159,7 +159,7 @@ public class PartitionReplicaListener implements
ReplicaListener {
private final Lazy<TableSchemaAwareIndexStorage> pkIndexStorage;
/** Secondary indices. */
- private final Supplier<Map<UUID, TableSchemaAwareIndexStorage>>
secondaryIndexStorages;
+ private final Supplier<Map<Integer, TableSchemaAwareIndexStorage>>
secondaryIndexStorages;
/** Versioned partition storage. */
private final MvPartitionStorage mvDataStorage;
@@ -203,7 +203,7 @@ public class PartitionReplicaListener implements
ReplicaListener {
*/
private final ConcurrentHashMap<UUID, CompletableFuture<TxMeta>>
txTimestampUpdateMap = new ConcurrentHashMap<>();
- private final Supplier<Map<UUID, IndexLocker>> indexesLockers;
+ private final Supplier<Map<Integer, IndexLocker>> indexesLockers;
private final ConcurrentMap<UUID, TxCleanupReadyFutureList>
txCleanupReadyFutures = new ConcurrentHashMap<>();
@@ -259,9 +259,9 @@ public class PartitionReplicaListener implements
ReplicaListener {
Executor scanRequestExecutor,
int partId,
int tableId,
- Supplier<Map<UUID, IndexLocker>> indexesLockers,
+ Supplier<Map<Integer, IndexLocker>> indexesLockers,
Lazy<TableSchemaAwareIndexStorage> pkIndexStorage,
- Supplier<Map<UUID, TableSchemaAwareIndexStorage>>
secondaryIndexStorages,
+ Supplier<Map<Integer, TableSchemaAwareIndexStorage>>
secondaryIndexStorages,
HybridClock hybridClock,
PendingComparableValuesTracker<HybridTimestamp, Void> safeTime,
TxStateStorage txStateStorage,
@@ -745,7 +745,7 @@ public class PartitionReplicaListener implements
ReplicaListener {
IgniteUuid cursorId = new IgniteUuid(txId, request.scanId());
- UUID indexId = request.indexToUse();
+ Integer indexId = request.indexToUse();
BinaryTuple exactKey = request.exactKey();
@@ -782,7 +782,7 @@ public class PartitionReplicaListener implements
ReplicaListener {
IgniteUuid cursorId = new IgniteUuid(txId, request.scanId());
- UUID indexId = request.indexToUse();
+ Integer indexId = request.indexToUse();
BinaryTuplePrefix lowerBound = request.lowerBound();
BinaryTuplePrefix upperBound = request.upperBound();
@@ -2402,7 +2402,7 @@ public class PartitionReplicaListener implements
ReplicaListener {
// Let's try to build an index for the previously created indexes
for the table.
TablesView tablesView =
mvTableStorage.tablesConfiguration().value();
- for (UUID indexId : collectIndexIds(tablesView)) {
+ for (int indexId : collectIndexIds(tablesView)) {
startBuildIndex(createIndexDescriptor(tablesView, indexId));
}
});
@@ -2479,11 +2479,11 @@ public class PartitionReplicaListener implements
ReplicaListener {
indexBuilder.startBuildIndex(tableId(), partId(),
indexDescriptor.id(), indexStorage, mvDataStorage, raftClient);
}
- private List<UUID> collectIndexIds(TablesView tablesView) {
+ private int[] collectIndexIds(TablesView tablesView) {
return tablesView.indexes().stream()
.filter(tableIndexView -> replicationGroupId.tableId() ==
tableIndexView.tableId())
- .map(TableIndexView::id)
- .collect(toList());
+ .mapToInt(TableIndexView::id)
+ .toArray();
}
private int partId() {
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 4d7f369aee..444799efd9 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
@@ -89,7 +89,6 @@ import org.apache.ignite.lang.IgniteStringFormatter;
import org.apache.ignite.lang.IgniteTetraFunction;
import org.apache.ignite.network.ClusterNode;
import org.apache.ignite.tx.TransactionException;
-import org.jetbrains.annotations.NotNull;
import org.jetbrains.annotations.Nullable;
import org.jetbrains.annotations.TestOnly;
@@ -361,11 +360,11 @@ public class InternalTableImpl implements InternalTable {
* @return Batch of retrieved rows.
*/
private CompletableFuture<Collection<BinaryRow>> enlistCursorInTx(
- @NotNull InternalTransaction tx,
+ InternalTransaction tx,
int partId,
long scanId,
int batchSize,
- @Nullable UUID indexId,
+ @Nullable Integer indexId,
@Nullable BinaryTuple exactKey,
@Nullable BinaryTuplePrefix lowerBound,
@Nullable BinaryTuplePrefix upperBound,
@@ -524,12 +523,11 @@ public class InternalTableImpl implements InternalTable {
}
}
- /** {@inheritDoc} */
@Override
public CompletableFuture<BinaryRow> get(
BinaryRowEx keyRow,
- @NotNull HybridTimestamp readTimestamp,
- @NotNull ClusterNode recipientNode
+ HybridTimestamp readTimestamp,
+ ClusterNode recipientNode
) {
int partId = partitionId(keyRow);
ReplicationGroupId partGroupId =
raftGroupServiceByPartitionId.get(partId).groupId();
@@ -575,8 +573,8 @@ public class InternalTableImpl implements InternalTable {
@Override
public CompletableFuture<Collection<BinaryRow>> getAll(
Collection<BinaryRowEx> keyRows,
- @NotNull HybridTimestamp readTimestamp,
- @NotNull ClusterNode recipientNode
+ HybridTimestamp readTimestamp,
+ ClusterNode recipientNode
) {
Int2ObjectOpenHashMap<List<BinaryRow>> keyRowsByPartition =
mapRowsToPartitions(keyRows);
@@ -838,39 +836,36 @@ public class InternalTableImpl implements InternalTable {
this::collectMultiRowsResponses);
}
- /** {@inheritDoc} */
@Override
public Publisher<BinaryRow> lookup(
int partId,
- @NotNull HybridTimestamp readTimestamp,
- @NotNull ClusterNode recipientNode,
- @NotNull UUID indexId,
+ HybridTimestamp readTimestamp,
+ ClusterNode recipientNode,
+ int indexId,
BinaryTuple key,
@Nullable BitSet columnsToInclude
) {
return scan(partId, readTimestamp, recipientNode, indexId, key, null,
null, 0, columnsToInclude);
}
- /** {@inheritDoc} */
@Override
public Publisher<BinaryRow> lookup(
int partId,
UUID txId,
PrimaryReplica recipient,
- UUID indexId,
+ int indexId,
BinaryTuple key,
@Nullable BitSet columnsToInclude
) {
return scan(partId, txId, recipient, indexId, key, null, null, 0,
columnsToInclude);
}
- /** {@inheritDoc} */
@Override
public Publisher<BinaryRow> scan(
int partId,
- @NotNull HybridTimestamp readTimestamp,
- @NotNull ClusterNode recipientNode,
- @Nullable UUID indexId,
+ HybridTimestamp readTimestamp,
+ ClusterNode recipientNode,
+ @Nullable Integer indexId,
@Nullable BinaryTuplePrefix lowerBound,
@Nullable BinaryTuplePrefix upperBound,
int flags,
@@ -881,9 +876,9 @@ public class InternalTableImpl implements InternalTable {
private Publisher<BinaryRow> scan(
int partId,
- @NotNull HybridTimestamp readTimestamp,
- @NotNull ClusterNode recipientNode,
- @Nullable UUID indexId,
+ HybridTimestamp readTimestamp,
+ ClusterNode recipientNode,
+ @Nullable Integer indexId,
@Nullable BinaryTuple exactKey,
@Nullable BinaryTuplePrefix lowerBound,
@Nullable BinaryTuplePrefix upperBound,
@@ -919,12 +914,11 @@ public class InternalTableImpl implements InternalTable {
Function.identity());
}
- /** {@inheritDoc} */
@Override
public Publisher<BinaryRow> scan(
int partId,
@Nullable InternalTransaction tx,
- @Nullable UUID indexId,
+ @Nullable Integer indexId,
@Nullable BinaryTuplePrefix lowerBound,
@Nullable BinaryTuplePrefix upperBound,
int flags,
@@ -936,7 +930,7 @@ public class InternalTableImpl implements InternalTable {
private Publisher<BinaryRow> scan(
int partId,
@Nullable InternalTransaction tx,
- @Nullable UUID indexId,
+ @Nullable Integer indexId,
@Nullable BinaryTuple exactKey,
@Nullable BinaryTuplePrefix lowerBound,
@Nullable BinaryTuplePrefix upperBound,
@@ -979,13 +973,12 @@ public class InternalTableImpl implements InternalTable {
}
- /** {@inheritDoc} */
@Override
public Publisher<BinaryRow> scan(
int partId,
UUID txId,
PrimaryReplica recipient,
- @Nullable UUID indexId,
+ @Nullable Integer indexId,
@Nullable BinaryTuplePrefix lowerBound,
@Nullable BinaryTuplePrefix upperBound,
int flags,
@@ -998,7 +991,7 @@ public class InternalTableImpl implements InternalTable {
int partId,
UUID txId,
PrimaryReplica recipient,
- @Nullable UUID indexId,
+ @Nullable Integer indexId,
@Nullable BinaryTuple exactKey,
@Nullable BinaryTuplePrefix lowerBound,
@Nullable BinaryTuplePrefix upperBound,
diff --git
a/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/IndexBaseTest.java
b/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/IndexBaseTest.java
index da0be154a6..bced8e1d97 100644
---
a/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/IndexBaseTest.java
+++
b/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/IndexBaseTest.java
@@ -89,9 +89,9 @@ public abstract class IndexBaseTest extends
BaseMvStoragesTest {
@BeforeEach
void setUp(@InjectConfiguration DataStorageConfiguration dsCfg) {
- UUID pkIndexId = UUID.randomUUID();
- UUID sortedIndexId = UUID.randomUUID();
- UUID hashIndexId = UUID.randomUUID();
+ int pkIndexId = 1;
+ int sortedIndexId = 2;
+ int hashIndexId = 3;
pkInnerStorage = new TestHashIndexStorage(PARTITION_ID, null);
@@ -125,7 +125,7 @@ public abstract class IndexBaseTest extends
BaseMvStoragesTest {
storage = new TestMvPartitionStorage(PARTITION_ID);
- Map<UUID, TableSchemaAwareIndexStorage> indexes = Map.of(
+ Map<Integer, TableSchemaAwareIndexStorage> indexes = Map.of(
pkIndexId, pkStorage,
sortedIndexId, sortedIndexStorage,
hashIndexId, hashIndexStorage
diff --git
a/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/index/IndexUpdateHandlerTest.java
b/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/index/IndexUpdateHandlerTest.java
index f5e69837ed..7960b1f8b7 100644
---
a/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/index/IndexUpdateHandlerTest.java
+++
b/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/index/IndexUpdateHandlerTest.java
@@ -23,7 +23,6 @@ import static org.mockito.Mockito.verify;
import static org.mockito.Mockito.when;
import java.util.Map;
-import java.util.UUID;
import java.util.stream.Stream;
import org.apache.ignite.internal.schema.BinaryRow;
import org.apache.ignite.internal.storage.BinaryRowAndRowId;
@@ -43,7 +42,7 @@ public class IndexUpdateHandlerTest {
void testBuildIndex() {
TableSchemaAwareIndexStorage indexStorage = createIndexStorage();
- UUID indexId = UUID.randomUUID();
+ int indexId = 1;
TableIndexStoragesSupplier indexes =
mock(TableIndexStoragesSupplier.class);
diff --git
a/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/raft/PartitionCommandListenerTest.java
b/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/raft/PartitionCommandListenerTest.java
index 59fa8ca452..78adaee7e9 100644
---
a/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/raft/PartitionCommandListenerTest.java
+++
b/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/raft/PartitionCommandListenerTest.java
@@ -139,7 +139,7 @@ public class PartitionCommandListenerTest {
/** Primary index. */
private final TableSchemaAwareIndexStorage pkStorage = new
TableSchemaAwareIndexStorage(
- UUID.randomUUID(),
+ 1,
new TestHashIndexStorage(PARTITION_ID, null),
BinaryRowConverter.keyExtractor(SCHEMA)
);
@@ -482,7 +482,7 @@ public class PartitionCommandListenerTest {
@Test
void testBuildIndexCommand() {
- UUID indexId = UUID.randomUUID();
+ int indexId = 1;
doNothing().when(indexUpdateHandler).buildIndex(eq(indexId),
any(Stream.class), any());
@@ -509,7 +509,7 @@ public class PartitionCommandListenerTest {
inOrder.verify(partitionDataStorage, never()).lastApplied(5, 1);
}
- private BuildIndexCommand createBuildIndexCommand(UUID indexId, List<UUID>
rowUuids, boolean finish) {
+ private BuildIndexCommand createBuildIndexCommand(int indexId, List<UUID>
rowUuids, boolean finish) {
return msgFactory.buildIndexCommand()
.tablePartitionId(
msgFactory.tablePartitionIdMessage()
diff --git
a/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/replication/PartitionReplicaListenerIndexLockingTest.java
b/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/replication/PartitionReplicaListenerIndexLockingTest.java
index 402ec89e30..aa1c957354 100644
---
a/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/replication/PartitionReplicaListenerIndexLockingTest.java
+++
b/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/replication/PartitionReplicaListenerIndexLockingTest.java
@@ -30,6 +30,7 @@ import java.util.Iterator;
import java.util.List;
import java.util.Locale;
import java.util.Map;
+import java.util.Objects;
import java.util.UUID;
import java.util.concurrent.CompletableFuture;
import java.util.function.Function;
@@ -105,9 +106,9 @@ import org.junit.jupiter.params.provider.MethodSource;
public class PartitionReplicaListenerIndexLockingTest extends
IgniteAbstractTest {
private static final int PART_ID = 0;
private static final int TABLE_ID = 1;
- private static final UUID PK_INDEX_ID = new UUID(0L, 1L);
- private static final UUID HASH_INDEX_ID = new UUID(0L, 2L);
- private static final UUID SORTED_INDEX_ID = new UUID(0L, 3L);
+ private static final int PK_INDEX_ID = 1;
+ private static final int HASH_INDEX_ID = 2;
+ private static final int SORTED_INDEX_ID = 3;
private static final UUID TRANSACTION_ID =
TestTransactionIds.newTransactionId();
private static final HybridClock CLOCK = new HybridClockImpl();
private static final LockManager LOCK_MANAGER = new HeapLockManager();
@@ -262,17 +263,17 @@ public class PartitionReplicaListenerIndexLockingTest
extends IgniteAbstractTest
allOf(
hasItem(lockThat(
arg.expectedLockOnUniqueHash + " on unique
hash index",
- lock ->
PK_INDEX_ID.equals(lock.lockKey().contextId())
+ lock -> Objects.equals(PK_INDEX_ID,
lock.lockKey().contextId())
&& lock.lockMode() ==
arg.expectedLockOnUniqueHash
)),
hasItem(lockThat(
arg.expectedLockOnNonUniqueHash + " on non
unique hash index",
- lock ->
HASH_INDEX_ID.equals(lock.lockKey().contextId())
+ lock -> Objects.equals(HASH_INDEX_ID,
lock.lockKey().contextId())
&& lock.lockMode() ==
arg.expectedLockOnNonUniqueHash
)),
hasItem(lockThat(
arg.expectedLockOnSort + " on sorted index",
- lock ->
SORTED_INDEX_ID.equals(lock.lockKey().contextId())
+ lock -> Objects.equals(SORTED_INDEX_ID,
lock.lockKey().contextId())
&& lock.lockMode() ==
arg.expectedLockOnSort
))
)
@@ -309,19 +310,19 @@ public class PartitionReplicaListenerIndexLockingTest
extends IgniteAbstractTest
allOf(
hasItem(lockThat(
arg.expectedLockOnUniqueHash + " on unique
hash index",
- lock ->
PK_INDEX_ID.equals(lock.lockKey().contextId())
+ lock -> Objects.equals(PK_INDEX_ID,
lock.lockKey().contextId())
&&
row2HashKeyConverter.apply(row).byteBuffer().equals(lock.lockKey().key())
&& lock.lockMode() ==
arg.expectedLockOnUniqueHash
)),
hasItem(lockThat(
arg.expectedLockOnNonUniqueHash + " on non
unique hash index",
- lock ->
HASH_INDEX_ID.equals(lock.lockKey().contextId())
+ lock -> Objects.equals(HASH_INDEX_ID,
lock.lockKey().contextId())
&&
row2HashKeyConverter.apply(row).byteBuffer().equals(lock.lockKey().key())
&& lock.lockMode() ==
arg.expectedLockOnNonUniqueHash
)),
hasItem(lockThat(
arg.expectedLockOnSort + " on sorted
index",
- lock ->
SORTED_INDEX_ID.equals(lock.lockKey().contextId())
+ lock -> Objects.equals(SORTED_INDEX_ID,
lock.lockKey().contextId())
&&
row2SortKeyConverter.apply(row).byteBuffer().equals(lock.lockKey().key())
&& lock.lockMode() ==
arg.expectedLockOnSort
))
diff --git
a/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/replication/PartitionReplicaListenerTest.java
b/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/replication/PartitionReplicaListenerTest.java
index 6ba6468f4a..78e3c27990 100644
---
a/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/replication/PartitionReplicaListenerTest.java
+++
b/modules/table/src/test/java/org/apache/ignite/internal/table/distributed/replication/PartitionReplicaListenerTest.java
@@ -346,9 +346,9 @@ public class PartitionReplicaListenerTest extends
IgniteAbstractTest {
lenient().when(schemas.waitForSchemasAvailability(any())).thenReturn(completedFuture(null));
lenient().when(schemas.waitForSchemaAvailability(anyInt(),
anyInt())).thenReturn(completedFuture(null));
- UUID pkIndexId = UUID.randomUUID();
- UUID sortedIndexId = UUID.randomUUID();
- UUID hashIndexId = UUID.randomUUID();
+ int pkIndexId = 1;
+ int sortedIndexId = 2;
+ int hashIndexId = 3;
schemaDescriptor = schemaDescriptorWith(CURRENT_SCHEMA_VERSION);
schemaDescriptorVersion2 = schemaDescriptorWith(FUTURE_SCHEMA_VERSION);
@@ -621,7 +621,7 @@ public class PartitionReplicaListenerTest extends
IgniteAbstractTest {
@Test
public void testWriteScanRetrieveBatchReplicaRequestWithSortedIndex()
throws Exception {
UUID txId = TestTransactionIds.newTransactionId();
- UUID sortedIndexId = sortedIndexStorage.id();
+ int sortedIndexId = sortedIndexStorage.id();
IntStream.range(0, 6).forEach(i -> {
RowId rowId = new RowId(partId);
@@ -728,7 +728,7 @@ public class PartitionReplicaListenerTest extends
IgniteAbstractTest {
@Test
public void testReadOnlyScanRetrieveBatchReplicaRequestSortedIndex()
throws Exception {
UUID txId = TestTransactionIds.newTransactionId();
- UUID sortedIndexId = sortedIndexStorage.id();
+ int sortedIndexId = sortedIndexStorage.id();
IntStream.range(0, 6).forEach(i -> {
RowId rowId = new RowId(partId);
@@ -830,7 +830,7 @@ public class PartitionReplicaListenerTest extends
IgniteAbstractTest {
@Test
public void testReadOnlyScanRetrieveBatchReplicaRequstHashIndex() throws
Exception {
UUID txId = TestTransactionIds.newTransactionId();
- UUID hashIndexId = hashIndexStorage.id();
+ int hashIndexId = hashIndexStorage.id();
IntStream.range(0, 7).forEach(i -> {
RowId rowId = new RowId(partId);
diff --git
a/modules/table/src/testFixtures/java/org/apache/ignite/internal/table/impl/DummyInternalTableImpl.java
b/modules/table/src/testFixtures/java/org/apache/ignite/internal/table/impl/DummyInternalTableImpl.java
index b5c36b28d3..2fa6a01114 100644
---
a/modules/table/src/testFixtures/java/org/apache/ignite/internal/table/impl/DummyInternalTableImpl.java
+++
b/modules/table/src/testFixtures/java/org/apache/ignite/internal/table/impl/DummyInternalTableImpl.java
@@ -261,7 +261,7 @@ public class DummyInternalTableImpl extends
InternalTableImpl {
).when(svc).run(any());
int tableId = tableId();
- UUID indexId = UUID.randomUUID();
+ int indexId = 1;
Function<BinaryRow, BinaryTuple> row2Tuple =
BinaryRowConverter.keyExtractor(schema);
@@ -429,15 +429,15 @@ public class DummyInternalTableImpl extends
InternalTableImpl {
*
* @param indexes Index storage by ID.
*/
- public static TableIndexStoragesSupplier
createTableIndexStoragesSupplier(Map<UUID, TableSchemaAwareIndexStorage>
indexes) {
+ public static TableIndexStoragesSupplier
createTableIndexStoragesSupplier(Map<Integer, TableSchemaAwareIndexStorage>
indexes) {
return new TableIndexStoragesSupplier() {
@Override
- public Map<UUID, TableSchemaAwareIndexStorage> get() {
+ public Map<Integer, TableSchemaAwareIndexStorage> get() {
return indexes;
}
@Override
- public void addIndexToWaitIfAbsent(UUID indexId) {
+ public void addIndexToWaitIfAbsent(int indexId) {
fail("not supported");
}
};