This is an automated email from the ASF dual-hosted git repository.
ashapkin 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 6c8ede20815 IGNITE-27279 Change default partition count (#7564)
6c8ede20815 is described below
commit 6c8ede20815b4c389d0c0503d69264d826a83ff7
Author: Mikhail Efremov <[email protected]>
AuthorDate: Tue Mar 3 19:23:01 2026 +0600
IGNITE-27279 Change default partition count (#7564)
---
.../compaction/AbstractCatalogCompactionTest.java | 4 +-
.../catalog/{it => }/ItConcurrentDdlsTest.java | 2 +-
.../catalog/ItDefaultPartitionCountTest.java | 77 +++++++++++++
.../internal/catalog/CatalogManagerImpl.java | 12 +-
.../PartitionCountCalculationParameters.java | 2 +-
...Provider.java => PartitionCountCalculator.java} | 22 +++-
.../ignite/internal/catalog/UpdateContext.java | 20 ++--
.../internal/catalog/commands/CatalogUtils.java | 13 ++-
.../catalog/commands/CreateTableCommand.java | 2 +-
.../catalog/commands/CreateZoneCommand.java | 8 +-
.../internal/catalog/CatalogManagerSelfTest.java | 4 +-
.../internal/catalog/BaseCatalogManagerTest.java | 2 +-
.../ignite/internal/catalog/CatalogTestUtils.java | 18 ++-
.../ignite/internal/cli/CliIntegrationTest.java | 1 +
.../partitions/reset/ItResetPartitionsTest.java | 8 +-
.../restart/ItRestartPartitionsTest.java | 10 +-
.../partitions/states/ItPartitionStatesTest.java | 32 ++++--
.../compute/events/ItComputeEventsTest.java | 9 +-
.../internal/system/CpuInformationProvider.java} | 14 ++-
...niteDistributionZoneManagerNodeRestartTest.java | 4 +-
.../rebalance/ItRebalanceDistributedTest.java | 4 +-
.../distributionzones/DataNodesManager.java | 2 +-
.../distributionzones/DistributionZoneManager.java | 4 +
.../distributionzones/DistributionZonesUtil.java | 38 +++++--
.../DistributionZoneStorageProfilesFilterTest.java | 15 ++-
modules/partition-replicator/build.gradle | 1 +
.../partition/replicator/fixtures/Node.java | 4 +-
.../PartitionReplicaLifecycleManagerTest.java | 4 +-
.../internal/placementdriver/LeaseUpdater.java | 1 -
.../org/apache/ignite/raft/ItRaftMetricTest.java | 5 +
.../rest/metrics/ItMetricControllerTest.java | 1 +
...terRecoveryControllerRestartPartitionsTest.java | 8 +-
...ControllerRestartPartitionsWithCleanupTest.java | 8 +-
.../recovery/ItDisasterRecoveryControllerTest.java | 40 +++++--
modules/runner/build.gradle | 1 +
.../runner/app/ItIgniteNodeRestartTest.java | 4 +-
.../runner/app/PlatformTestNodeRunner.java | 9 ++
.../org/apache/ignite/internal/app/IgniteImpl.java | 31 +++++-
.../DataNodesAwarePartitionCountCalculator.java | 66 +++++++++++
.../catalog/EstimatedDataNodeCountProvider.java} | 17 +--
...r.java => PartitionCountCalculatorWrapper.java} | 21 ++--
.../system/JvmCpuInformationProvider.java} | 12 +-
...DataNodesAwarePartitionCountCalculatorTest.java | 124 +++++++++++++++++++++
...va => PartitionCountCalculatorWrapperTest.java} | 36 +++---
.../system/JvmCpuInformationProviderTest.java | 44 ++++++++
.../internal/ClusterPerClassIntegrationTest.java | 50 ++++++++-
.../internal/ClusterPerTestIntegrationTest.java | 31 +++++-
...tSchemaForwardCompatibilityConsistencyTest.java | 32 +++++-
modules/sql-engine/build.gradle | 2 +
.../internal/sql/api/ItSqlCreateZoneTest.java | 5 +
.../engine/systemviews/AbstractSystemViewTest.java | 4 +
.../engine/systemviews/ItZonesSystemViewTest.java | 13 ++-
.../internal/sql/sqllogic/ItSqlLogicTest.java | 12 ++
...ilablePartitionsRecoveryByFilterUpdateTest.java | 1 +
.../disaster/ItDisasterRecoveryManagerTest.java | 10 +-
55 files changed, 759 insertions(+), 165 deletions(-)
diff --git
a/modules/catalog-compaction/src/test/java/org/apache/ignite/internal/catalog/compaction/AbstractCatalogCompactionTest.java
b/modules/catalog-compaction/src/test/java/org/apache/ignite/internal/catalog/compaction/AbstractCatalogCompactionTest.java
index 81cdaabac1d..4dea99a8808 100644
---
a/modules/catalog-compaction/src/test/java/org/apache/ignite/internal/catalog/compaction/AbstractCatalogCompactionTest.java
+++
b/modules/catalog-compaction/src/test/java/org/apache/ignite/internal/catalog/compaction/AbstractCatalogCompactionTest.java
@@ -26,7 +26,7 @@ import static org.mockito.Mockito.spy;
import java.util.List;
import java.util.concurrent.ScheduledExecutorService;
import org.apache.ignite.internal.catalog.CatalogManagerImpl;
-import org.apache.ignite.internal.catalog.PartitionCountProvider;
+import org.apache.ignite.internal.catalog.CatalogTestUtils;
import org.apache.ignite.internal.catalog.storage.UpdateLogImpl;
import org.apache.ignite.internal.failure.FailureProcessor;
import org.apache.ignite.internal.failure.NoOpFailureManager;
@@ -86,7 +86,7 @@ abstract class AbstractCatalogCompactionTest extends
BaseIgniteAbstractTest {
clockService,
failureProcessor,
() -> TEST_DELAY_DURATION,
- PartitionCountProvider.defaultPartitionCountProvider()
+ CatalogTestUtils.defaultPartitionCountCalculator()
);
assertThat(startAsync(new ComponentContext(), metastore),
willCompleteSuccessfully());
diff --git
a/modules/catalog/src/integrationTest/java/org/apache/ignite/internal/catalog/it/ItConcurrentDdlsTest.java
b/modules/catalog/src/integrationTest/java/org/apache/ignite/internal/catalog/ItConcurrentDdlsTest.java
similarity index 98%
rename from
modules/catalog/src/integrationTest/java/org/apache/ignite/internal/catalog/it/ItConcurrentDdlsTest.java
rename to
modules/catalog/src/integrationTest/java/org/apache/ignite/internal/catalog/ItConcurrentDdlsTest.java
index 9d7f33c9577..78bec198ef3 100644
---
a/modules/catalog/src/integrationTest/java/org/apache/ignite/internal/catalog/it/ItConcurrentDdlsTest.java
+++
b/modules/catalog/src/integrationTest/java/org/apache/ignite/internal/catalog/ItConcurrentDdlsTest.java
@@ -15,7 +15,7 @@
* limitations under the License.
*/
-package org.apache.ignite.internal.catalog.it;
+package org.apache.ignite.internal.catalog;
import static java.util.concurrent.TimeUnit.SECONDS;
import static org.junit.jupiter.api.Assertions.assertDoesNotThrow;
diff --git
a/modules/catalog/src/integrationTest/java/org/apache/ignite/internal/catalog/ItDefaultPartitionCountTest.java
b/modules/catalog/src/integrationTest/java/org/apache/ignite/internal/catalog/ItDefaultPartitionCountTest.java
new file mode 100644
index 00000000000..3a7317c26de
--- /dev/null
+++
b/modules/catalog/src/integrationTest/java/org/apache/ignite/internal/catalog/ItDefaultPartitionCountTest.java
@@ -0,0 +1,77 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.catalog;
+
+import static org.apache.ignite.internal.TestWrappers.unwrapIgniteImpl;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+
+import org.apache.ignite.internal.ClusterPerTestIntegrationTest;
+import org.apache.ignite.internal.app.IgniteImpl;
+import org.apache.ignite.internal.catalog.commands.CatalogUtils;
+import org.apache.ignite.internal.catalog.descriptors.CatalogZoneDescriptor;
+import org.apache.ignite.internal.system.CpuInformationProvider;
+import org.apache.ignite.internal.system.JvmCpuInformationProvider;
+import org.junit.jupiter.api.Test;
+
+class ItDefaultPartitionCountTest extends ClusterPerTestIntegrationTest {
+ private static final CpuInformationProvider cpuInfoProvider = new
JvmCpuInformationProvider();
+
+ @Override
+ protected boolean shouldCreateDefaultZone() {
+ return false;
+ }
+
+ @Test
+ public void defaultPartitionCountTest() {
+ // Triggers lazy default zone creation.
+ sql("CREATE TABLE IF NOT EXISTS test_table (id INT, val VARCHAR,
PRIMARY KEY (id))");
+
+ assertEquals(partitionCount(CatalogUtils.DEFAULT_REPLICA_COUNT),
zoneDescriptor(CatalogUtils.DEFAULT_ZONE_NAME).partitions());
+
+ sql("CREATE ZONE TEST_ZONE_R1 WITH REPLICAS=1, STORAGE_PROFILES='" +
CatalogService.DEFAULT_STORAGE_PROFILE + "'");
+
+ assertEquals(partitionCount(1),
zoneDescriptor("TEST_ZONE_R1").partitions());
+
+ sql("CREATE ZONE TEST_ZONE_R3 WITH REPLICAS=3, STORAGE_PROFILES='" +
CatalogService.DEFAULT_STORAGE_PROFILE + "'");
+
+ assertEquals(partitionCount(3),
zoneDescriptor("TEST_ZONE_R3").partitions());
+
+ // 23 is just some simple number.
+ sql("CREATE ZONE TEST_ZONE_P23 WITH PARTITIONS=23, REPLICAS=3,
STORAGE_PROFILES='" + CatalogService.DEFAULT_STORAGE_PROFILE + "'");
+
+ assertEquals(23, zoneDescriptor("TEST_ZONE_P23").partitions());
+ }
+
+ private int partitionCount(int replicas) {
+ return initialNodes()
+ * cpuInfoProvider.availableProcessors()
+ * DataNodesAwarePartitionCountCalculator.SCALE_FACTOR
+ / replicas;
+ }
+
+ private CatalogZoneDescriptor zoneDescriptor(String zoneName) {
+ IgniteImpl node = unwrapIgniteImpl(node(0));
+
+ CatalogZoneDescriptor descriptor =
node.catalogManager().latestCatalog().zone(zoneName);
+
+ assertNotNull(descriptor);
+
+ return descriptor;
+ }
+}
diff --git
a/modules/catalog/src/main/java/org/apache/ignite/internal/catalog/CatalogManagerImpl.java
b/modules/catalog/src/main/java/org/apache/ignite/internal/catalog/CatalogManagerImpl.java
index f7fcbe32cd8..827d4b82d6e 100644
---
a/modules/catalog/src/main/java/org/apache/ignite/internal/catalog/CatalogManagerImpl.java
+++
b/modules/catalog/src/main/java/org/apache/ignite/internal/catalog/CatalogManagerImpl.java
@@ -114,9 +114,9 @@ public class CatalogManagerImpl extends
AbstractEventProducer<CatalogEvent, Cata
private final Object lastSaveUpdateFutureMutex = new Object();
/**
- * Partition count provider for command update contexts.
+ * Partition count calculator for command update contexts.
*/
- private final PartitionCountProvider partitionCountProvider;
+ private final PartitionCountCalculator partitionCountCalculator;
/**
* Constructor.
@@ -126,14 +126,14 @@ public class CatalogManagerImpl extends
AbstractEventProducer<CatalogEvent, Cata
ClockService clockService,
FailureProcessor failureProcessor,
LongSupplier delayDurationMsSupplier,
- PartitionCountProvider partitionCountProvider
+ PartitionCountCalculator partitionCountCalculator
) {
this.updateLog = updateLog;
this.clockService = clockService;
this.failureProcessor = failureProcessor;
this.delayDurationMsSupplier = delayDurationMsSupplier;
this.catalogSystemViewProvider = new CatalogSystemViewRegistry(() ->
catalogAt(clockService.nowLong()));
- this.partitionCountProvider = partitionCountProvider;
+ this.partitionCountCalculator = partitionCountCalculator;
}
@Override
@@ -267,7 +267,7 @@ public class CatalogManagerImpl extends
AbstractEventProducer<CatalogEvent, Cata
var initialEntries = new ArrayList<UpdateEntry>();
- var context = new UpdateContext(emptyCatalog, partitionCountProvider);
+ var context = new UpdateContext(emptyCatalog,
partitionCountCalculator);
for (UpdateProducer producer : initCommands) {
List<UpdateEntry> entries = producer.get(context);
@@ -420,7 +420,7 @@ public class CatalogManagerImpl extends
AbstractEventProducer<CatalogEvent, Cata
BitSet applyResults = new BitSet(updateProducers.size());
List<UpdateEntry> bulkUpdateEntries = new ArrayList<>();
try {
- UpdateContext updateContext = new UpdateContext(catalog,
partitionCountProvider);
+ UpdateContext updateContext = new UpdateContext(catalog,
partitionCountCalculator);
for (int i = 0; i < updateProducers.size(); i++) {
UpdateProducer update = updateProducers.get(i);
List<UpdateEntry> entries = update.get(updateContext);
diff --git
a/modules/catalog/src/main/java/org/apache/ignite/internal/catalog/PartitionCountCalculationParameters.java
b/modules/catalog/src/main/java/org/apache/ignite/internal/catalog/PartitionCountCalculationParameters.java
index 2d1e9aaa6f7..ce174d0d8b5 100644
---
a/modules/catalog/src/main/java/org/apache/ignite/internal/catalog/PartitionCountCalculationParameters.java
+++
b/modules/catalog/src/main/java/org/apache/ignite/internal/catalog/PartitionCountCalculationParameters.java
@@ -29,7 +29,7 @@ import java.util.List;
import org.jetbrains.annotations.Nullable;
/**
- * Parameters container for {@link PartitionCountProvider#calculate}.
+ * Parameters container for {@link PartitionCountCalculator#calculate}.
*/
public final class PartitionCountCalculationParameters {
private final String dataNodesFilter;
diff --git
a/modules/catalog/src/main/java/org/apache/ignite/internal/catalog/PartitionCountProvider.java
b/modules/catalog/src/main/java/org/apache/ignite/internal/catalog/PartitionCountCalculator.java
similarity index 52%
copy from
modules/catalog/src/main/java/org/apache/ignite/internal/catalog/PartitionCountProvider.java
copy to
modules/catalog/src/main/java/org/apache/ignite/internal/catalog/PartitionCountCalculator.java
index 7029cc53987..f5faed046d4 100644
---
a/modules/catalog/src/main/java/org/apache/ignite/internal/catalog/PartitionCountProvider.java
+++
b/modules/catalog/src/main/java/org/apache/ignite/internal/catalog/PartitionCountCalculator.java
@@ -17,16 +17,26 @@
package org.apache.ignite.internal.catalog;
-import static
org.apache.ignite.internal.catalog.commands.CatalogUtils.DEFAULT_PARTITION_COUNT;
-
/**
- * Default partition count provider.
+ * Function that calculates how much partitions a zone should use in case
partition count parameter wasn't specified by a user.
*/
@FunctionalInterface
-public interface PartitionCountProvider {
+public interface PartitionCountCalculator {
+ /**
+ * Calculates partition count depends on the given parameters set.
+ *
+ * @param params Container of parameters that may be used to calculate
partition count.
+ * @return Partition count for a zone.
+ */
int calculate(PartitionCountCalculationParameters params);
- static PartitionCountProvider defaultPartitionCountProvider() {
- return params -> DEFAULT_PARTITION_COUNT;
+ /**
+ * Creates a partition calculator that always return the given partitions
count value.
+ *
+ * @param partitions Partitions count to return on every {@link
PartitionCountCalculator#calculate} call regardless of passed params.
+ * @return Partition count that equals to the given one value.
+ */
+ static PartitionCountCalculator staticPartitionCountCalculator(int
partitions) {
+ return params -> partitions;
}
}
diff --git
a/modules/catalog/src/main/java/org/apache/ignite/internal/catalog/UpdateContext.java
b/modules/catalog/src/main/java/org/apache/ignite/internal/catalog/UpdateContext.java
index 3557ae055f5..21edfae6a3c 100644
---
a/modules/catalog/src/main/java/org/apache/ignite/internal/catalog/UpdateContext.java
+++
b/modules/catalog/src/main/java/org/apache/ignite/internal/catalog/UpdateContext.java
@@ -18,6 +18,7 @@
package org.apache.ignite.internal.catalog;
import java.util.function.Function;
+import org.apache.ignite.internal.catalog.commands.CatalogUtils;
/**
* Context contains two instances of the catalog: the base one and the updated
one.
@@ -28,24 +29,29 @@ import java.util.function.Function;
* processing of the current batch of commands.
*/
public class UpdateContext {
+ /** Static calculator to use in case if the calculation function wasn't
specified on construction. */
+ private static final PartitionCountCalculator STATIC_PARTITION_CALCULATOR
= PartitionCountCalculator.staticPartitionCountCalculator(
+ CatalogUtils.DEFAULT_PARTITION_COUNT
+ );
+
/** The base catalog descriptor. */
private final Catalog baseCatalog;
/** The updatable catalog descriptor. */
private Catalog updatableCatalog;
- private final PartitionCountProvider partitionCountProvider;
+ private final PartitionCountCalculator partitionCountCalculator;
/** Constructor. */
public UpdateContext(Catalog catalog) {
- this(catalog, PartitionCountProvider.defaultPartitionCountProvider());
+ this(catalog, STATIC_PARTITION_CALCULATOR);
}
/** Constructor. */
- public UpdateContext(Catalog catalog, PartitionCountProvider
partitionCountProvider) {
+ public UpdateContext(Catalog catalog, PartitionCountCalculator
partitionCountCalculator) {
this.baseCatalog = catalog;
this.updatableCatalog = catalog;
- this.partitionCountProvider = partitionCountProvider;
+ this.partitionCountCalculator = partitionCountCalculator;
}
/**
@@ -70,8 +76,8 @@ public class UpdateContext {
updatableCatalog = updater.apply(updatableCatalog);
}
- /** Returns partition count provider. */
- public PartitionCountProvider partitionCountProvider() {
- return partitionCountProvider;
+ /** Returns partition count calculator. */
+ public PartitionCountCalculator partitionCountCalculator() {
+ return partitionCountCalculator;
}
}
diff --git
a/modules/catalog/src/main/java/org/apache/ignite/internal/catalog/commands/CatalogUtils.java
b/modules/catalog/src/main/java/org/apache/ignite/internal/catalog/commands/CatalogUtils.java
index 986c4308d13..ab3e903cf3d 100644
---
a/modules/catalog/src/main/java/org/apache/ignite/internal/catalog/commands/CatalogUtils.java
+++
b/modules/catalog/src/main/java/org/apache/ignite/internal/catalog/commands/CatalogUtils.java
@@ -41,7 +41,7 @@ import org.apache.ignite.internal.catalog.Catalog;
import org.apache.ignite.internal.catalog.CatalogValidationException;
import org.apache.ignite.internal.catalog.IndexNotFoundValidationException;
import org.apache.ignite.internal.catalog.PartitionCountCalculationParameters;
-import org.apache.ignite.internal.catalog.PartitionCountProvider;
+import org.apache.ignite.internal.catalog.PartitionCountCalculator;
import org.apache.ignite.internal.catalog.commands.DefaultValue.FunctionCall;
import org.apache.ignite.internal.catalog.commands.DefaultValue.Type;
import org.apache.ignite.internal.catalog.descriptors.CatalogIndexDescriptor;
@@ -591,14 +591,14 @@ public final class CatalogUtils {
*/
public static CatalogZoneDescriptor createDefaultZoneDescriptor(
Catalog catalog,
- PartitionCountProvider partitionCountProvider,
+ PartitionCountCalculator partitionCountCalculator,
int newDefaultZoneId,
Collection<UpdateEntry> updateEntries
) throws CatalogValidationException {
// TODO: Remove after
https://issues.apache.org/jira/browse/IGNITE-26798
checkDuplicateDefaultZoneName(catalog);
- CatalogZoneDescriptor defaultZone =
createDefaultZoneDescriptor(partitionCountProvider, newDefaultZoneId);
+ CatalogZoneDescriptor defaultZone =
createDefaultZoneDescriptor(partitionCountCalculator, newDefaultZoneId);
updateEntries.add(new NewZoneEntry(defaultZone));
updateEntries.add(new SetDefaultZoneEntry(defaultZone.id()));
@@ -606,14 +606,17 @@ public final class CatalogUtils {
return defaultZone;
}
- private static CatalogZoneDescriptor
createDefaultZoneDescriptor(PartitionCountProvider partitionCountProvider, int
newDefaultZoneId) {
+ private static CatalogZoneDescriptor createDefaultZoneDescriptor(
+ PartitionCountCalculator partitionCountCalculator,
+ int newDefaultZoneId
+ ) {
PartitionCountCalculationParameters
partitionCountCalculationParameters =
PartitionCountCalculationParameters.builder()
.replicaFactor(DEFAULT_REPLICA_COUNT)
.dataNodesFilter(DEFAULT_FILTER)
.storageProfiles(List.of(DEFAULT_STORAGE_PROFILE))
.build();
- int partitionCount =
partitionCountProvider.calculate(partitionCountCalculationParameters);
+ int partitionCount =
partitionCountCalculator.calculate(partitionCountCalculationParameters);
return new CatalogZoneDescriptor(
newDefaultZoneId,
diff --git
a/modules/catalog/src/main/java/org/apache/ignite/internal/catalog/commands/CreateTableCommand.java
b/modules/catalog/src/main/java/org/apache/ignite/internal/catalog/commands/CreateTableCommand.java
index 1162fd61f13..4994d72ebee 100644
---
a/modules/catalog/src/main/java/org/apache/ignite/internal/catalog/commands/CreateTableCommand.java
+++
b/modules/catalog/src/main/java/org/apache/ignite/internal/catalog/commands/CreateTableCommand.java
@@ -143,7 +143,7 @@ public class CreateTableCommand extends
AbstractTableCommand {
List<UpdateEntry> updateEntries = new ArrayList<>(5);
CatalogZoneDescriptor zone = shouldCreateNewDefaultZone(catalog,
zoneName)
- ? createDefaultZoneDescriptor(catalog,
updateContext.partitionCountProvider(), id++, updateEntries)
+ ? createDefaultZoneDescriptor(catalog,
updateContext.partitionCountCalculator(), id++, updateEntries)
: zoneByNameOrDefaultOrThrow(catalog, zoneName);
assert zone != null;
diff --git
a/modules/catalog/src/main/java/org/apache/ignite/internal/catalog/commands/CreateZoneCommand.java
b/modules/catalog/src/main/java/org/apache/ignite/internal/catalog/commands/CreateZoneCommand.java
index c3a4b9bd065..b5c445d3e98 100644
---
a/modules/catalog/src/main/java/org/apache/ignite/internal/catalog/commands/CreateZoneCommand.java
+++
b/modules/catalog/src/main/java/org/apache/ignite/internal/catalog/commands/CreateZoneCommand.java
@@ -41,7 +41,7 @@ import org.apache.ignite.internal.catalog.Catalog;
import org.apache.ignite.internal.catalog.CatalogCommand;
import org.apache.ignite.internal.catalog.CatalogValidationException;
import org.apache.ignite.internal.catalog.PartitionCountCalculationParameters;
-import org.apache.ignite.internal.catalog.PartitionCountProvider;
+import org.apache.ignite.internal.catalog.PartitionCountCalculator;
import org.apache.ignite.internal.catalog.UpdateContext;
import
org.apache.ignite.internal.catalog.descriptors.CatalogStorageProfileDescriptor;
import
org.apache.ignite.internal.catalog.descriptors.CatalogStorageProfilesDescriptor;
@@ -134,7 +134,7 @@ public class CreateZoneCommand extends AbstractZoneCommand {
throw
duplicateDistributionZoneNameCatalogValidationException(zoneName);
}
- CatalogZoneDescriptor zoneDesc =
descriptor(updateContext.partitionCountProvider(), catalog.objectIdGenState());
+ CatalogZoneDescriptor zoneDesc =
descriptor(updateContext.partitionCountCalculator(),
catalog.objectIdGenState());
return List.of(
new NewZoneEntry(zoneDesc),
@@ -142,7 +142,7 @@ public class CreateZoneCommand extends AbstractZoneCommand {
);
}
- private CatalogZoneDescriptor descriptor(PartitionCountProvider
partitionCountProvider, int objectId) {
+ private CatalogZoneDescriptor descriptor(PartitionCountCalculator
partitionCountCalculator, int objectId) {
String filter = requireNonNullElse(this.filter, DEFAULT_FILTER);
int replicas = requireNonNullElse(this.replicas,
DEFAULT_REPLICA_COUNT);
@@ -163,7 +163,7 @@ public class CreateZoneCommand extends AbstractZoneCommand {
return new CatalogZoneDescriptor(
objectId,
zoneName,
- requireNonNullElse(partitions,
partitionCountProvider.calculate(partitionCountCalculationParameters)),
+ requireNonNullElse(partitions,
partitionCountCalculator.calculate(partitionCountCalculationParameters)),
replicas,
requireNonNullElse(quorumSize, defaultQuorumSize(replicas)),
requireNonNullElse(
diff --git
a/modules/catalog/src/test/java/org/apache/ignite/internal/catalog/CatalogManagerSelfTest.java
b/modules/catalog/src/test/java/org/apache/ignite/internal/catalog/CatalogManagerSelfTest.java
index 50ecd0cc648..fb872e762e5 100644
---
a/modules/catalog/src/test/java/org/apache/ignite/internal/catalog/CatalogManagerSelfTest.java
+++
b/modules/catalog/src/test/java/org/apache/ignite/internal/catalog/CatalogManagerSelfTest.java
@@ -186,7 +186,7 @@ public class CatalogManagerSelfTest extends
BaseCatalogManagerTest {
clockService,
new NoOpFailureManager(),
delayDuration::get,
- PartitionCountProvider.defaultPartitionCountProvider()
+ CatalogTestUtils.defaultPartitionCountCalculator()
);
assertThat(manager.startAsync(componentContext),
willCompleteSuccessfully());
@@ -331,7 +331,7 @@ public class CatalogManagerSelfTest extends
BaseCatalogManagerTest {
clockService,
new NoOpFailureManager(),
delayDuration::get,
- PartitionCountProvider.defaultPartitionCountProvider()
+ CatalogTestUtils.defaultPartitionCountCalculator()
);
assertThat(manager.startAsync(startComponentContext),
willCompleteSuccessfully());
diff --git
a/modules/catalog/src/testFixtures/java/org/apache/ignite/internal/catalog/BaseCatalogManagerTest.java
b/modules/catalog/src/testFixtures/java/org/apache/ignite/internal/catalog/BaseCatalogManagerTest.java
index 387e11022b8..96d40398074 100644
---
a/modules/catalog/src/testFixtures/java/org/apache/ignite/internal/catalog/BaseCatalogManagerTest.java
+++
b/modules/catalog/src/testFixtures/java/org/apache/ignite/internal/catalog/BaseCatalogManagerTest.java
@@ -132,7 +132,7 @@ public abstract class BaseCatalogManagerTest extends
BaseIgniteAbstractTest {
clockService,
failureProcessor,
delayDuration::get,
- PartitionCountProvider.defaultPartitionCountProvider()
+ CatalogTestUtils.defaultPartitionCountCalculator()
);
ComponentContext context = new ComponentContext();
diff --git
a/modules/catalog/src/testFixtures/java/org/apache/ignite/internal/catalog/CatalogTestUtils.java
b/modules/catalog/src/testFixtures/java/org/apache/ignite/internal/catalog/CatalogTestUtils.java
index 1313a234195..b61dce0a0e3 100644
---
a/modules/catalog/src/testFixtures/java/org/apache/ignite/internal/catalog/CatalogTestUtils.java
+++
b/modules/catalog/src/testFixtures/java/org/apache/ignite/internal/catalog/CatalogTestUtils.java
@@ -19,6 +19,8 @@ package org.apache.ignite.internal.catalog;
import static java.util.concurrent.CompletableFuture.allOf;
import static
org.apache.ignite.internal.catalog.CatalogService.DEFAULT_STORAGE_PROFILE;
+import static
org.apache.ignite.internal.catalog.PartitionCountCalculator.staticPartitionCountCalculator;
+import static
org.apache.ignite.internal.catalog.commands.CatalogUtils.DEFAULT_PARTITION_COUNT;
import static
org.apache.ignite.internal.testframework.IgniteTestUtils.runAsync;
import static
org.apache.ignite.internal.testframework.matchers.CompletableFutureMatcher.willCompleteSuccessfully;
import static
org.apache.ignite.internal.util.CompletableFutures.falseCompletedFuture;
@@ -112,7 +114,7 @@ public class CatalogTestUtils {
clockService,
failureProcessor,
delayDurationMsSupplier,
- PartitionCountProvider.defaultPartitionCountProvider()
+ defaultPartitionCountCalculator()
) {
@Override
public CompletableFuture<Void> startAsync(ComponentContext
componentContext) {
@@ -164,7 +166,7 @@ public class CatalogTestUtils {
new TestClockService(clock, clockWaiter),
failureProcessor,
() -> TEST_DELAY_DURATION,
- PartitionCountProvider.defaultPartitionCountProvider()
+ defaultPartitionCountCalculator()
) {
@Override
public CompletableFuture<Void> startAsync(ComponentContext
componentContext) {
@@ -210,7 +212,7 @@ public class CatalogTestUtils {
new TestClockService(clock, clockWaiter),
failureProcessor,
() -> TEST_DELAY_DURATION,
- PartitionCountProvider.defaultPartitionCountProvider()
+ defaultPartitionCountCalculator()
);
}
@@ -248,7 +250,7 @@ public class CatalogTestUtils {
new TestClockService(clock, clockWaiter),
failureProcessor,
delayDurationMsSupplier,
- PartitionCountProvider.defaultPartitionCountProvider()
+ defaultPartitionCountCalculator()
) {
@Override
public CompletableFuture<Void> startAsync(ComponentContext
componentContext) {
@@ -320,7 +322,7 @@ public class CatalogTestUtils {
new TestClockService(clock, clockWaiter),
failureProcessor,
() -> TEST_DELAY_DURATION,
- PartitionCountProvider.defaultPartitionCountProvider()
+ defaultPartitionCountCalculator()
) {
@Override
public CompletableFuture<Void> startAsync(ComponentContext
componentContext) {
@@ -368,7 +370,7 @@ public class CatalogTestUtils {
new TestClockService(clock, clockWaiter),
new NoOpFailureManager(),
() -> TEST_DELAY_DURATION,
- PartitionCountProvider.defaultPartitionCountProvider()
+ defaultPartitionCountCalculator()
) {
@Override
public CompletableFuture<Void> startAsync(ComponentContext
componentContext) {
@@ -503,6 +505,10 @@ public class CatalogTestUtils {
return AlterZoneCommand.builder().zoneName(zoneName);
}
+ public static PartitionCountCalculator defaultPartitionCountCalculator() {
+ return staticPartitionCountCalculator(DEFAULT_PARTITION_COUNT);
+ }
+
private static class TestUpdateLog implements UpdateLog {
private final HybridClock clock;
diff --git
a/modules/cli/src/integrationTest/java/org/apache/ignite/internal/cli/CliIntegrationTest.java
b/modules/cli/src/integrationTest/java/org/apache/ignite/internal/cli/CliIntegrationTest.java
index b94292ffc2c..b2ab7350714 100644
---
a/modules/cli/src/integrationTest/java/org/apache/ignite/internal/cli/CliIntegrationTest.java
+++
b/modules/cli/src/integrationTest/java/org/apache/ignite/internal/cli/CliIntegrationTest.java
@@ -83,6 +83,7 @@ public abstract class CliIntegrationTest extends
ClusterPerClassIntegrationTest
new MetricSource().name("transactions").enabled(true),
new MetricSource().name("placement-driver").enabled(true),
new MetricSource().name("resource.vacuum").enabled(true),
+ new MetricSource().name("zones.Default").enabled(true),
new MetricSource().name("clock.service").enabled(true),
new MetricSource().name("index.builder").enabled(true),
new MetricSource().name("raft.snapshots").enabled(true),
diff --git
a/modules/cli/src/integrationTest/java/org/apache/ignite/internal/cli/commands/recovery/partitions/reset/ItResetPartitionsTest.java
b/modules/cli/src/integrationTest/java/org/apache/ignite/internal/cli/commands/recovery/partitions/reset/ItResetPartitionsTest.java
index 064596f9e78..145430651ea 100644
---
a/modules/cli/src/integrationTest/java/org/apache/ignite/internal/cli/commands/recovery/partitions/reset/ItResetPartitionsTest.java
+++
b/modules/cli/src/integrationTest/java/org/apache/ignite/internal/cli/commands/recovery/partitions/reset/ItResetPartitionsTest.java
@@ -83,15 +83,17 @@ public abstract class ItResetPartitionsTest extends
CliIntegrationTest {
@Test
public void testResetPartitionsPartitionsOutOfRange() {
+ int partitionCount = partitionsCount(ZONE);
+
execute(CLUSTER_URL_OPTION, NODE_URL,
RECOVERY_ZONE_NAME_OPTION, ZONE,
- RECOVERY_PARTITION_IDS_OPTION,
String.valueOf(DEFAULT_PARTITION_COUNT));
+ RECOVERY_PARTITION_IDS_OPTION, String.valueOf(partitionCount));
assertErrOutputContains(String.format(
"Partition IDs should be in range [0, %d] for zone %s, found:
%d",
- DEFAULT_PARTITION_COUNT - 1,
+ partitionCount - 1,
ZONE,
- DEFAULT_PARTITION_COUNT
+ partitionCount
));
assertOutputIsEmpty();
}
diff --git
a/modules/cli/src/integrationTest/java/org/apache/ignite/internal/cli/commands/recovery/partitions/restart/ItRestartPartitionsTest.java
b/modules/cli/src/integrationTest/java/org/apache/ignite/internal/cli/commands/recovery/partitions/restart/ItRestartPartitionsTest.java
index f716f81609c..e76b5f5cc6a 100644
---
a/modules/cli/src/integrationTest/java/org/apache/ignite/internal/cli/commands/recovery/partitions/restart/ItRestartPartitionsTest.java
+++
b/modules/cli/src/integrationTest/java/org/apache/ignite/internal/cli/commands/recovery/partitions/restart/ItRestartPartitionsTest.java
@@ -37,8 +37,6 @@ public abstract class ItRestartPartitionsTest extends
CliIntegrationTest {
private static final String TABLE_NAME = "first_ZONE_table";
- private static final int DEFAULT_PARTITION_COUNT = 25;
-
@BeforeAll
public void createTables() {
sql(String.format("CREATE ZONE \"%s\" (REPLICAS %s) storage profiles
['%s']",
@@ -133,17 +131,19 @@ public abstract class ItRestartPartitionsTest extends
CliIntegrationTest {
@Test
public void testRestartPartitionsPartitionsOutOfRange() {
+ int partitionCount = partitionsCount(ZONE);
+
execute(CLUSTER_URL_OPTION, NODE_URL,
RECOVERY_ZONE_NAME_OPTION, ZONE,
- RECOVERY_PARTITION_IDS_OPTION,
String.valueOf(DEFAULT_PARTITION_COUNT)
+ RECOVERY_PARTITION_IDS_OPTION, String.valueOf(partitionCount)
);
assertOutputIsEmpty();
assertErrOutputContains(String.format(
"Partition IDs should be in range [0, %d] for zone %s, found:
%d",
- DEFAULT_PARTITION_COUNT - 1,
+ partitionCount - 1,
ZONE,
- DEFAULT_PARTITION_COUNT
+ partitionCount
));
}
diff --git
a/modules/cli/src/integrationTest/java/org/apache/ignite/internal/cli/commands/recovery/partitions/states/ItPartitionStatesTest.java
b/modules/cli/src/integrationTest/java/org/apache/ignite/internal/cli/commands/recovery/partitions/states/ItPartitionStatesTest.java
index d1c7c480e3c..0294de76946 100644
---
a/modules/cli/src/integrationTest/java/org/apache/ignite/internal/cli/commands/recovery/partitions/states/ItPartitionStatesTest.java
+++
b/modules/cli/src/integrationTest/java/org/apache/ignite/internal/cli/commands/recovery/partitions/states/ItPartitionStatesTest.java
@@ -40,9 +40,8 @@ import org.junit.jupiter.params.provider.ValueSource;
/** Base test class for Cluster Recovery partition states commands. */
// TODO IGNITE-23617 refactor to use more flexible output matching.
-
public abstract class ItPartitionStatesTest extends CliIntegrationTest {
- private static final int DEFAULT_PARTITION_COUNT = 25;
+ private static final int PARTITIONS_COUNT = 10;
private static final Set<String> ZONES = Set.of("first_ZONE",
"second_ZONE", "third_ZONE");
@@ -67,11 +66,21 @@ public abstract class ItPartitionStatesTest extends
CliIntegrationTest {
@BeforeAll
public static void createTables() {
ZONES_CONTAINING_TABLES.forEach(name -> {
- sql(String.format("CREATE ZONE \"%s\" storage profiles ['%s']",
name, DEFAULT_AIPERSIST_PROFILE_NAME));
+ sql(String.format(
+ "CREATE ZONE \"%s\" (PARTITIONS %d) storage profiles
['%s']",
+ name,
+ PARTITIONS_COUNT,
+ DEFAULT_AIPERSIST_PROFILE_NAME
+ ));
sql(String.format("CREATE TABLE \"%s_table\" (id INT PRIMARY KEY,
val INT) ZONE \"%1$s\"", name));
});
- sql(String.format("CREATE ZONE \"%s\" storage profiles ['%s']",
EMPTY_ZONE, DEFAULT_AIPERSIST_PROFILE_NAME));
+ sql(String.format(
+ "CREATE ZONE \"%s\" (PARTITIONS %d) storage profiles ['%s']",
+ EMPTY_ZONE,
+ PARTITIONS_COUNT,
+ DEFAULT_AIPERSIST_PROFILE_NAME
+ ));
nodeNames = CLUSTER.runningNodes().map(Ignite::name).collect(toSet());
}
@@ -83,7 +92,7 @@ public abstract class ItPartitionStatesTest extends
CliIntegrationTest {
global ? RECOVERY_PARTITION_GLOBAL_OPTION :
RECOVERY_PARTITION_LOCAL_OPTION,
PLAIN_OPTION);
- checkOutput(global, ZONES_CONTAINING_TABLES, nodeNames,
DEFAULT_PARTITION_COUNT);
+ checkOutput(global, ZONES_CONTAINING_TABLES, nodeNames,
PARTITIONS_COUNT);
}
@ParameterizedTest
@@ -95,7 +104,7 @@ public abstract class ItPartitionStatesTest extends
CliIntegrationTest {
PLAIN_OPTION
);
- checkOutput(global, ZONES, nodeNames, DEFAULT_PARTITION_COUNT);
+ checkOutput(global, ZONES, nodeNames, PARTITIONS_COUNT);
}
@ParameterizedTest
@@ -151,7 +160,7 @@ public abstract class ItPartitionStatesTest extends
CliIntegrationTest {
PLAIN_OPTION
);
- checkOutput(global, MIXED_CASE_ZONES, nodeNames,
DEFAULT_PARTITION_COUNT);
+ checkOutput(global, MIXED_CASE_ZONES, nodeNames, PARTITIONS_COUNT);
}
@ParameterizedTest
@@ -189,9 +198,12 @@ public abstract class ItPartitionStatesTest extends
CliIntegrationTest {
@ParameterizedTest
@ValueSource(booleans = {false, true})
void testLocalPartitionStatesPartitionOutOfRange(boolean global) {
- String partitions = "0,1," + DEFAULT_PARTITION_COUNT;
String zoneName = ZONES_CONTAINING_TABLES.stream().findAny().get();
+ int partitionCount = partitionsCount(zoneName);
+
+ String partitions = "0,1," + partitionCount;
+
execute(CLUSTER_URL_OPTION, NODE_URL,
RECOVERY_PARTITION_IDS_OPTION, partitions,
RECOVERY_ZONE_NAMES_OPTION, zoneName,
@@ -201,9 +213,9 @@ public abstract class ItPartitionStatesTest extends
CliIntegrationTest {
assertErrOutputContains(String.format(
"Partition IDs should be in range [0, %d] for zone %s, found:
%d",
- DEFAULT_PARTITION_COUNT - 1,
+ partitionCount - 1,
zoneName,
- DEFAULT_PARTITION_COUNT
+ partitionCount
));
assertOutputIsEmpty();
diff --git
a/modules/compute/src/integrationTest/java/org/apache/ignite/internal/compute/events/ItComputeEventsTest.java
b/modules/compute/src/integrationTest/java/org/apache/ignite/internal/compute/events/ItComputeEventsTest.java
index 0fe1036a00d..9e7e163792c 100644
---
a/modules/compute/src/integrationTest/java/org/apache/ignite/internal/compute/events/ItComputeEventsTest.java
+++
b/modules/compute/src/integrationTest/java/org/apache/ignite/internal/compute/events/ItComputeEventsTest.java
@@ -19,6 +19,7 @@ package org.apache.ignite.internal.compute.events;
import static org.apache.ignite.compute.JobStatus.CANCELED;
import static org.apache.ignite.compute.JobStatus.EXECUTING;
+import static org.apache.ignite.internal.TestWrappers.unwrapIgniteImpl;
import static
org.apache.ignite.internal.compute.events.ComputeEventMetadata.Type.BROADCAST;
import static
org.apache.ignite.internal.compute.events.ComputeEventMetadata.Type.MAP_REDUCE;
import static
org.apache.ignite.internal.compute.events.ComputeEventMetadata.Type.SINGLE;
@@ -45,6 +46,7 @@ import static org.hamcrest.Matchers.hasSize;
import static org.hamcrest.Matchers.in;
import static org.hamcrest.Matchers.notNullValue;
import static org.hamcrest.Matchers.nullValue;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
import com.fasterxml.jackson.core.JsonProcessingException;
import com.fasterxml.jackson.databind.ObjectMapper;
@@ -68,6 +70,7 @@ import org.apache.ignite.compute.task.MapReduceTask;
import org.apache.ignite.compute.task.TaskExecution;
import org.apache.ignite.internal.ClusterPerClassIntegrationTest;
import org.apache.ignite.internal.ConfigOverride;
+import org.apache.ignite.internal.catalog.descriptors.CatalogZoneDescriptor;
import org.apache.ignite.internal.compute.events.ComputeEventMetadata.Type;
import org.apache.ignite.internal.compute.events.EventMatcher.Event;
import org.apache.ignite.internal.compute.utils.InteractiveJobs;
@@ -184,7 +187,11 @@ abstract class ItComputeEventsTest extends
ClusterPerClassIntegrationTest {
assertThat(broadcastExecution.resultsAsync(),
willCompleteSuccessfully());
- int defaultPartitionCount = 25;
+ CatalogZoneDescriptor defaultZoneDesc =
unwrapIgniteImpl(CLUSTER.aliveNode()).catalogManager().latestCatalog().defaultZone();
+
+ assertNotNull(defaultZoneDesc);
+
+ int defaultPartitionCount = defaultZoneDesc.partitions();
assertThat(broadcastExecution.executions(),
hasSize(defaultPartitionCount));
await().until(logInspector::events, hasSize(defaultPartitionCount *
3));
diff --git
a/modules/sql-engine/src/integrationTest/java/org/apache/ignite/internal/sql/engine/systemviews/AbstractSystemViewTest.java
b/modules/core/src/main/java/org/apache/ignite/internal/system/CpuInformationProvider.java
similarity index 70%
copy from
modules/sql-engine/src/integrationTest/java/org/apache/ignite/internal/sql/engine/systemviews/AbstractSystemViewTest.java
copy to
modules/core/src/main/java/org/apache/ignite/internal/system/CpuInformationProvider.java
index 25c717d2a9f..4bc4628b6fd 100644
---
a/modules/sql-engine/src/integrationTest/java/org/apache/ignite/internal/sql/engine/systemviews/AbstractSystemViewTest.java
+++
b/modules/core/src/main/java/org/apache/ignite/internal/system/CpuInformationProvider.java
@@ -15,12 +15,16 @@
* limitations under the License.
*/
-package org.apache.ignite.internal.sql.engine.systemviews;
-
-import org.apache.ignite.internal.sql.engine.BaseSqlMultiStatementTest;
+package org.apache.ignite.internal.system;
/**
- * Base class for SQL system views integration tests.
+ * Provides information about local node CPU hardware.
*/
-public class AbstractSystemViewTest extends BaseSqlMultiStatementTest {
+public interface CpuInformationProvider {
+ /**
+ * Calculates and returns available number of hardware CPU cores of the
local node machine.
+ *
+ * @return CPU count of the local node.
+ */
+ int availableProcessors();
}
diff --git
a/modules/distribution-zones/src/integrationTest/java/org/apache/ignite/internal/distributionzones/ItIgniteDistributionZoneManagerNodeRestartTest.java
b/modules/distribution-zones/src/integrationTest/java/org/apache/ignite/internal/distributionzones/ItIgniteDistributionZoneManagerNodeRestartTest.java
index 3c6842a1be4..22946cdfab2 100644
---
a/modules/distribution-zones/src/integrationTest/java/org/apache/ignite/internal/distributionzones/ItIgniteDistributionZoneManagerNodeRestartTest.java
+++
b/modules/distribution-zones/src/integrationTest/java/org/apache/ignite/internal/distributionzones/ItIgniteDistributionZoneManagerNodeRestartTest.java
@@ -82,7 +82,7 @@ import org.apache.ignite.configuration.validation.Validator;
import org.apache.ignite.internal.BaseIgniteRestartTest;
import org.apache.ignite.internal.catalog.CatalogManager;
import org.apache.ignite.internal.catalog.CatalogManagerImpl;
-import org.apache.ignite.internal.catalog.PartitionCountProvider;
+import org.apache.ignite.internal.catalog.CatalogTestUtils;
import org.apache.ignite.internal.catalog.descriptors.CatalogZoneDescriptor;
import org.apache.ignite.internal.catalog.descriptors.ConsistencyMode;
import org.apache.ignite.internal.catalog.storage.UpdateLogImpl;
@@ -316,7 +316,7 @@ public class ItIgniteDistributionZoneManagerNodeRestartTest
extends BaseIgniteRe
clockService,
failureProcessor,
() -> TEST_DELAY_DURATION,
- PartitionCountProvider.defaultPartitionCountProvider()
+ CatalogTestUtils.defaultPartitionCountCalculator()
);
LowWatermark lowWatermark = mock(LowWatermark.class);
diff --git
a/modules/distribution-zones/src/integrationTest/java/org/apache/ignite/internal/rebalance/ItRebalanceDistributedTest.java
b/modules/distribution-zones/src/integrationTest/java/org/apache/ignite/internal/rebalance/ItRebalanceDistributedTest.java
index d4087f63c43..e410bbeb1b4 100644
---
a/modules/distribution-zones/src/integrationTest/java/org/apache/ignite/internal/rebalance/ItRebalanceDistributedTest.java
+++
b/modules/distribution-zones/src/integrationTest/java/org/apache/ignite/internal/rebalance/ItRebalanceDistributedTest.java
@@ -105,7 +105,7 @@ import
org.apache.ignite.configuration.validation.ConfigurationValidationExcepti
import org.apache.ignite.internal.app.ThreadPoolsManager;
import org.apache.ignite.internal.catalog.CatalogManager;
import org.apache.ignite.internal.catalog.CatalogManagerImpl;
-import org.apache.ignite.internal.catalog.PartitionCountProvider;
+import org.apache.ignite.internal.catalog.CatalogTestUtils;
import org.apache.ignite.internal.catalog.commands.ColumnParams;
import org.apache.ignite.internal.catalog.storage.UpdateLogImpl;
import org.apache.ignite.internal.cluster.management.ClusterIdHolder;
@@ -1498,7 +1498,7 @@ public class ItRebalanceDistributedTest extends
BaseIgniteAbstractTest {
clockService,
failureManager,
delayDurationMsSupplier,
- PartitionCountProvider.defaultPartitionCountProvider()
+ CatalogTestUtils.defaultPartitionCountCalculator()
);
indexMetaStorage = new IndexMetaStorage(catalogManager,
lowWatermark, metaStorageManager);
diff --git
a/modules/distribution-zones/src/main/java/org/apache/ignite/internal/distributionzones/DataNodesManager.java
b/modules/distribution-zones/src/main/java/org/apache/ignite/internal/distributionzones/DataNodesManager.java
index 9f7da26a2fd..9746e4633dd 100644
---
a/modules/distribution-zones/src/main/java/org/apache/ignite/internal/distributionzones/DataNodesManager.java
+++
b/modules/distribution-zones/src/main/java/org/apache/ignite/internal/distributionzones/DataNodesManager.java
@@ -1010,7 +1010,7 @@ public class DataNodesManager {
.build();
}
- private Set<NodeWithAttributes> topologyNodes() {
+ Set<NodeWithAttributes> topologyNodes() {
// It means that the zone was created but the data nodes value had not
been updated yet.
// So the data nodes value will be equals to the logical topology on
the descLastUpdateRevision.
Entry topologyEntry =
metaStorageManager.getLocally(zonesLogicalTopologyKey());
diff --git
a/modules/distribution-zones/src/main/java/org/apache/ignite/internal/distributionzones/DistributionZoneManager.java
b/modules/distribution-zones/src/main/java/org/apache/ignite/internal/distributionzones/DistributionZoneManager.java
index 029374a6589..97147f9faa8 100644
---
a/modules/distribution-zones/src/main/java/org/apache/ignite/internal/distributionzones/DistributionZoneManager.java
+++
b/modules/distribution-zones/src/main/java/org/apache/ignite/internal/distributionzones/DistributionZoneManager.java
@@ -344,6 +344,10 @@ public class DistributionZoneManager extends
return nullCompletedFuture();
}
+ public int estimatedDataNodesCount(String dataNodeFilter, List<String>
storageProfiles) {
+ return filterDataNodes(dataNodesManager.topologyNodes(),
dataNodeFilter, storageProfiles).size();
+ }
+
/**
* Returns data nodes at the current time.
*
diff --git
a/modules/distribution-zones/src/main/java/org/apache/ignite/internal/distributionzones/DistributionZonesUtil.java
b/modules/distribution-zones/src/main/java/org/apache/ignite/internal/distributionzones/DistributionZonesUtil.java
index 6f8e90f5202..f3dca8f83ca 100644
---
a/modules/distribution-zones/src/main/java/org/apache/ignite/internal/distributionzones/DistributionZonesUtil.java
+++
b/modules/distribution-zones/src/main/java/org/apache/ignite/internal/distributionzones/DistributionZonesUtil.java
@@ -491,22 +491,18 @@ public class DistributionZonesUtil {
* Filters storage profiles.
*
* @param node Node with storage profile attributes.
- * @param zoneStorageProfiles Zone's storage profiles.
+ * @param zoneStorageProfiles Zone's storage profile names.
* @return True, if matches, false otherwise.
*/
public static boolean filterStorageProfiles(
NodeWithAttributes node,
- List<CatalogStorageProfileDescriptor> zoneStorageProfiles
+ List<String> zoneStorageProfiles
) {
if (node.storageProfiles() == null) {
return false;
}
- List<String> zoneStorageProfilesNames = zoneStorageProfiles.stream()
- .map(CatalogStorageProfileDescriptor::storageProfile)
- .collect(toList());
-
- return new
HashSet<>(node.storageProfiles()).containsAll(zoneStorageProfilesNames);
+ return new
HashSet<>(node.storageProfiles()).containsAll(zoneStorageProfiles);
}
/**
@@ -519,10 +515,34 @@ public class DistributionZonesUtil {
public static Set<NodeWithAttributes> filterDataNodes(
Set<NodeWithAttributes> dataNodes,
CatalogZoneDescriptor zoneDescriptor
+ ) {
+ List<String> storageProfiles = zoneDescriptor
+ .storageProfiles()
+ .profiles()
+ .stream()
+ .map(CatalogStorageProfileDescriptor::storageProfile)
+ .collect(toList());
+
+ return filterDataNodes(dataNodes, zoneDescriptor.filter(),
storageProfiles);
+ }
+
+ /**
+ * Filters {@code dataNodes} according to the provided filter and storage
profiles from {@code zoneDescriptor}.
+ * Nodes' attributes and storage profiles are taken from {@code
nodesAttributes} map.
+ *
+ * @param dataNodes Data nodes.
+ * @param filter Data nodes filter.
+ * @param storageProfiles Storage profiles.
+ * @return Filtered data nodes.
+ */
+ public static Set<NodeWithAttributes> filterDataNodes(
+ Set<NodeWithAttributes> dataNodes,
+ String filter,
+ List<String> storageProfiles
) {
return dataNodes.stream()
- .filter(n -> filterNodeAttributes(n.userAttributes(),
zoneDescriptor.filter()))
- .filter(n -> filterStorageProfiles(n,
zoneDescriptor.storageProfiles().profiles()))
+ .filter(n -> filterNodeAttributes(n.userAttributes(), filter))
+ .filter(n -> filterStorageProfiles(n, storageProfiles))
.collect(toSet());
}
diff --git
a/modules/distribution-zones/src/test/java/org/apache/ignite/internal/distributionzones/DistributionZoneStorageProfilesFilterTest.java
b/modules/distribution-zones/src/test/java/org/apache/ignite/internal/distributionzones/DistributionZoneStorageProfilesFilterTest.java
index 13987279ba6..6ccd4d9c12a 100644
---
a/modules/distribution-zones/src/test/java/org/apache/ignite/internal/distributionzones/DistributionZoneStorageProfilesFilterTest.java
+++
b/modules/distribution-zones/src/test/java/org/apache/ignite/internal/distributionzones/DistributionZoneStorageProfilesFilterTest.java
@@ -18,7 +18,7 @@
package org.apache.ignite.internal.distributionzones;
import static java.util.UUID.randomUUID;
-import static
org.apache.ignite.internal.catalog.commands.CatalogUtils.fromParams;
+import static java.util.stream.Collectors.toList;
import static
org.apache.ignite.internal.distributionzones.DistributionZonesUtil.filterStorageProfiles;
import static
org.apache.ignite.internal.distributionzones.DistributionZonesUtil.parseStorageProfiles;
import static org.junit.jupiter.api.Assertions.assertFalse;
@@ -26,6 +26,7 @@ import static org.junit.jupiter.api.Assertions.assertTrue;
import java.util.List;
import java.util.Map;
+import org.apache.ignite.internal.catalog.commands.StorageProfileParams;
import org.junit.jupiter.api.Test;
/** Tests storage profiles filtering. */
@@ -37,7 +38,7 @@ public class DistributionZoneStorageProfilesFilterTest {
String zoneStorageProfiles = "qwe,asd";
- assertTrue(filterStorageProfiles(node,
fromParams(parseStorageProfiles(zoneStorageProfiles)).profiles()));
+ assertTrue(filterStorageProfiles(node,
storageProfilesFromParams(parseStorageProfiles(zoneStorageProfiles))));
}
@Test
@@ -47,7 +48,7 @@ public class DistributionZoneStorageProfilesFilterTest {
String zoneStorageProfiles = "qwe,asd";
- assertFalse(filterStorageProfiles(node,
fromParams(parseStorageProfiles(zoneStorageProfiles)).profiles()));
+ assertFalse(filterStorageProfiles(node,
storageProfilesFromParams(parseStorageProfiles(zoneStorageProfiles))));
}
@Test
@@ -57,7 +58,7 @@ public class DistributionZoneStorageProfilesFilterTest {
String zoneStorageProfiles = "zxc,asd";
- assertTrue(filterStorageProfiles(node,
fromParams(parseStorageProfiles(zoneStorageProfiles)).profiles()));
+ assertTrue(filterStorageProfiles(node,
storageProfilesFromParams(parseStorageProfiles(zoneStorageProfiles))));
}
@Test
@@ -67,6 +68,10 @@ public class DistributionZoneStorageProfilesFilterTest {
String zoneStorageProfiles = "zxc, asd";
- assertTrue(filterStorageProfiles(node,
fromParams(parseStorageProfiles(zoneStorageProfiles)).profiles()));
+ assertTrue(filterStorageProfiles(node,
storageProfilesFromParams(parseStorageProfiles(zoneStorageProfiles))));
+ }
+
+ private static List<String>
storageProfilesFromParams(List<StorageProfileParams> params) {
+ return
params.stream().map(StorageProfileParams::storageProfile).collect(toList());
}
}
diff --git a/modules/partition-replicator/build.gradle
b/modules/partition-replicator/build.gradle
index 105833a7f35..bd5deaaf666 100644
--- a/modules/partition-replicator/build.gradle
+++ b/modules/partition-replicator/build.gradle
@@ -64,6 +64,7 @@ dependencies {
testImplementation libs.awaitility
integrationTestImplementation
testFixtures(project(':ignite-cluster-management'))
+ integrationTestImplementation testFixtures(project(':ignite-catalog'))
integrationTestImplementation testFixtures(project(':ignite-core'))
integrationTestImplementation
testFixtures(project(':ignite-configuration'))
integrationTestImplementation
testFixtures(project(':ignite-distribution-zones'))
diff --git
a/modules/partition-replicator/src/integrationTest/java/org/apache/ignite/internal/partition/replicator/fixtures/Node.java
b/modules/partition-replicator/src/integrationTest/java/org/apache/ignite/internal/partition/replicator/fixtures/Node.java
index 35db27e0351..10a680a8640 100644
---
a/modules/partition-replicator/src/integrationTest/java/org/apache/ignite/internal/partition/replicator/fixtures/Node.java
+++
b/modules/partition-replicator/src/integrationTest/java/org/apache/ignite/internal/partition/replicator/fixtures/Node.java
@@ -55,7 +55,7 @@ import org.apache.ignite.internal.app.NodePropertiesImpl;
import org.apache.ignite.internal.app.ThreadPoolsManager;
import org.apache.ignite.internal.catalog.CatalogManager;
import org.apache.ignite.internal.catalog.CatalogManagerImpl;
-import org.apache.ignite.internal.catalog.PartitionCountProvider;
+import org.apache.ignite.internal.catalog.CatalogTestUtils;
import org.apache.ignite.internal.catalog.compaction.CatalogCompactionRunner;
import org.apache.ignite.internal.catalog.descriptors.CatalogTableDescriptor;
import org.apache.ignite.internal.catalog.descriptors.CatalogZoneDescriptor;
@@ -651,7 +651,7 @@ public class Node {
clockService,
failureManager,
delayDurationMsSupplier,
- PartitionCountProvider.defaultPartitionCountProvider()
+ CatalogTestUtils.defaultPartitionCountCalculator()
);
schemaManager = new SchemaManager(registry, catalogManager);
diff --git
a/modules/partition-replicator/src/test/java/org/apache/ignite/internal/partition/replicator/PartitionReplicaLifecycleManagerTest.java
b/modules/partition-replicator/src/test/java/org/apache/ignite/internal/partition/replicator/PartitionReplicaLifecycleManagerTest.java
index 108b6aed850..b229f25c47c 100644
---
a/modules/partition-replicator/src/test/java/org/apache/ignite/internal/partition/replicator/PartitionReplicaLifecycleManagerTest.java
+++
b/modules/partition-replicator/src/test/java/org/apache/ignite/internal/partition/replicator/PartitionReplicaLifecycleManagerTest.java
@@ -66,7 +66,7 @@ import java.util.stream.IntStream;
import org.apache.ignite.internal.catalog.CatalogManager;
import org.apache.ignite.internal.catalog.CatalogManagerImpl;
import org.apache.ignite.internal.catalog.CatalogService;
-import org.apache.ignite.internal.catalog.PartitionCountProvider;
+import org.apache.ignite.internal.catalog.CatalogTestUtils;
import org.apache.ignite.internal.catalog.storage.UpdateLogImpl;
import
org.apache.ignite.internal.cluster.management.ClusterManagementGroupManager;
import org.apache.ignite.internal.configuration.SystemDistributedConfiguration;
@@ -230,7 +230,7 @@ class PartitionReplicaLifecycleManagerTest extends
BaseIgniteAbstractTest {
clockService,
failureManager,
() -> TEST_DELAY_DURATION,
- PartitionCountProvider.defaultPartitionCountProvider()
+ CatalogTestUtils.defaultPartitionCountCalculator()
);
replicaManager = spy(new ReplicaManager(
diff --git
a/modules/placement-driver/src/main/java/org/apache/ignite/internal/placementdriver/LeaseUpdater.java
b/modules/placement-driver/src/main/java/org/apache/ignite/internal/placementdriver/LeaseUpdater.java
index a06a09e7810..b5cbf650144 100644
---
a/modules/placement-driver/src/main/java/org/apache/ignite/internal/placementdriver/LeaseUpdater.java
+++
b/modules/placement-driver/src/main/java/org/apache/ignite/internal/placementdriver/LeaseUpdater.java
@@ -394,7 +394,6 @@ public class LeaseUpdater {
failureProcessor.process(new FailureContext(e, "Error
occurred when updating the leases."));
if (e instanceof Error) {
- // TODO IGNITE-20368 The node should be halted in case
of an error here.
throw (Error) e;
}
} finally {
diff --git
a/modules/raft/src/integrationTest/java/org/apache/ignite/raft/ItRaftMetricTest.java
b/modules/raft/src/integrationTest/java/org/apache/ignite/raft/ItRaftMetricTest.java
index a0687d50e50..f34bf8da886 100644
---
a/modules/raft/src/integrationTest/java/org/apache/ignite/raft/ItRaftMetricTest.java
+++
b/modules/raft/src/integrationTest/java/org/apache/ignite/raft/ItRaftMetricTest.java
@@ -38,6 +38,11 @@ public class ItRaftMetricTest extends
ClusterPerClassIntegrationTest {
private static final int PARTITION_COUNT = 10;
+ @Override
+ protected boolean shouldCreateDefaultZone() {
+ return false;
+ }
+
@Override
protected void configureInitParameters(InitParametersBuilder builder) {
// To trigger zone's raft partitions destruction.
diff --git
a/modules/rest/src/integrationTest/java/org/apache/ignite/internal/rest/metrics/ItMetricControllerTest.java
b/modules/rest/src/integrationTest/java/org/apache/ignite/internal/rest/metrics/ItMetricControllerTest.java
index c621fd93783..3ead07d5d90 100644
---
a/modules/rest/src/integrationTest/java/org/apache/ignite/internal/rest/metrics/ItMetricControllerTest.java
+++
b/modules/rest/src/integrationTest/java/org/apache/ignite/internal/rest/metrics/ItMetricControllerTest.java
@@ -71,6 +71,7 @@ class ItMetricControllerTest extends
ClusterPerClassIntegrationTest {
new MetricSource("transactions", true),
new MetricSource("resource.vacuum", true),
new MetricSource("placement-driver", true),
+ new MetricSource("zones.Default", true),
new MetricSource("clock.service", true),
new MetricSource("index.builder", true),
new MetricSource("raft.snapshots", true),
diff --git
a/modules/rest/src/integrationTest/java/org/apache/ignite/internal/rest/recovery/ItDisasterRecoveryControllerRestartPartitionsTest.java
b/modules/rest/src/integrationTest/java/org/apache/ignite/internal/rest/recovery/ItDisasterRecoveryControllerRestartPartitionsTest.java
index c8eadf08494..f1cbf89bd8f 100644
---
a/modules/rest/src/integrationTest/java/org/apache/ignite/internal/rest/recovery/ItDisasterRecoveryControllerRestartPartitionsTest.java
+++
b/modules/rest/src/integrationTest/java/org/apache/ignite/internal/rest/recovery/ItDisasterRecoveryControllerRestartPartitionsTest.java
@@ -93,15 +93,17 @@ public class
ItDisasterRecoveryControllerRestartPartitionsTest extends ClusterPe
@Test
void testRestartPartitionsPartitionsOutOfRange() {
- MutableHttpRequest<?> post = restartPartitionsRequest(Set.of(),
FIRST_ZONE, Set.of(DEFAULT_PARTITION_COUNT));
+ int partitionCount = partitionsCount(FIRST_ZONE);
+
+ MutableHttpRequest<?> post = restartPartitionsRequest(Set.of(),
FIRST_ZONE, Set.of(partitionCount));
assertThrowsProblem(
() -> client.toBlocking().exchange(post),
isProblem().withStatus(BAD_REQUEST).withDetail(String.format(
"Partition IDs should be in range [0, %d] for zone %s,
found: %d",
- DEFAULT_PARTITION_COUNT - 1,
+ partitionCount - 1,
FIRST_ZONE,
- DEFAULT_PARTITION_COUNT
+ partitionCount
))
);
}
diff --git
a/modules/rest/src/integrationTest/java/org/apache/ignite/internal/rest/recovery/ItDisasterRecoveryControllerRestartPartitionsWithCleanupTest.java
b/modules/rest/src/integrationTest/java/org/apache/ignite/internal/rest/recovery/ItDisasterRecoveryControllerRestartPartitionsWithCleanupTest.java
index 573523e55c4..8d17bd76501 100644
---
a/modules/rest/src/integrationTest/java/org/apache/ignite/internal/rest/recovery/ItDisasterRecoveryControllerRestartPartitionsWithCleanupTest.java
+++
b/modules/rest/src/integrationTest/java/org/apache/ignite/internal/rest/recovery/ItDisasterRecoveryControllerRestartPartitionsWithCleanupTest.java
@@ -114,15 +114,17 @@ public class
ItDisasterRecoveryControllerRestartPartitionsWithCleanupTest extend
@Test
void testRestartPartitionsWithCleanupPartitionsOutOfRange() {
- MutableHttpRequest<?> post = restartPartitionsRequest(Set.of(),
FIRST_ZONE, Set.of(DEFAULT_PARTITION_COUNT));
+ int partitionCount = partitionsCount(FIRST_ZONE);
+
+ MutableHttpRequest<?> post = restartPartitionsRequest(Set.of(),
FIRST_ZONE, Set.of(partitionCount));
assertThrowsProblem(
() -> client1.toBlocking().exchange(post),
isProblem().withStatus(BAD_REQUEST).withDetail(String.format(
"Partition IDs should be in range [0, %d] for zone %s,
found: %d",
- DEFAULT_PARTITION_COUNT - 1,
+ partitionCount - 1,
FIRST_ZONE,
- DEFAULT_PARTITION_COUNT
+ partitionCount
))
);
}
diff --git
a/modules/rest/src/integrationTest/java/org/apache/ignite/internal/rest/recovery/ItDisasterRecoveryControllerTest.java
b/modules/rest/src/integrationTest/java/org/apache/ignite/internal/rest/recovery/ItDisasterRecoveryControllerTest.java
index c0484dfa503..9b1a780ff02 100644
---
a/modules/rest/src/integrationTest/java/org/apache/ignite/internal/rest/recovery/ItDisasterRecoveryControllerTest.java
+++
b/modules/rest/src/integrationTest/java/org/apache/ignite/internal/rest/recovery/ItDisasterRecoveryControllerTest.java
@@ -70,6 +70,8 @@ import org.junit.jupiter.api.Test;
public class ItDisasterRecoveryControllerTest extends
ClusterPerClassIntegrationTest {
private static final String NODE_URL = "http://localhost:" +
ClusterConfiguration.DEFAULT_BASE_HTTP_PORT;
+ private static final int PARTITIONS_COUNT = 10;
+
private static final String FIRST_ZONE = "first_ZONE";
private static final String QUALIFIED_TABLE_NAME = canonicalName("PUBLIC",
"first_ZONE_table");
@@ -96,11 +98,21 @@ public class ItDisasterRecoveryControllerTest extends
ClusterPerClassIntegration
@BeforeAll
public static void setUp() {
ZONES_CONTAINING_TABLES.forEach(name -> {
- sql(String.format("CREATE ZONE \"%s\" storage profiles ['%s']",
name, DEFAULT_AIPERSIST_PROFILE_NAME));
+ sql(String.format(
+ "CREATE ZONE \"%s\" (PARTITIONS %d) storage profiles
['%s']",
+ name,
+ PARTITIONS_COUNT,
+ DEFAULT_AIPERSIST_PROFILE_NAME
+ ));
sql(String.format("CREATE TABLE PUBLIC.\"%s_table\" (id INT
PRIMARY KEY, val INT) ZONE \"%1$s\"", name));
});
- sql(String.format("CREATE ZONE \"%s\" storage profiles ['%s']",
EMPTY_ZONE, DEFAULT_AIPERSIST_PROFILE_NAME));
+ sql(String.format(
+ "CREATE ZONE \"%s\" (PARTITIONS %d) storage profiles ['%s']",
+ EMPTY_ZONE,
+ PARTITIONS_COUNT,
+ DEFAULT_AIPERSIST_PROFILE_NAME
+ ));
nodeNames = CLUSTER.runningNodes().map(Ignite::name).collect(toSet());
}
@@ -156,15 +168,17 @@ public class ItDisasterRecoveryControllerTest extends
ClusterPerClassIntegration
void testLocalPartitionStatesPartitionOutOfRange() {
String zoneName = ZONES_CONTAINING_TABLES.stream().findAny().get();
+ int partitionCount = partitionsCount(zoneName);
+
assertThrowsProblem(
() -> client.toBlocking().exchange(
- String.format("%s?partitionIds=0,4,%d&zoneNames=%s",
localStatePath(), DEFAULT_PARTITION_COUNT, zoneName)
+ String.format("%s?partitionIds=0,4,%d&zoneNames=%s",
localStatePath(), partitionCount, zoneName)
),
isProblem().withStatus(BAD_REQUEST).withDetail(String.format(
"Partition IDs should be in range [0, %d] for zone %s,
found: %d",
- DEFAULT_PARTITION_COUNT - 1,
+ partitionCount - 1,
zoneName,
- DEFAULT_PARTITION_COUNT
+ partitionCount
))
);
}
@@ -302,17 +316,19 @@ public class ItDisasterRecoveryControllerTest extends
ClusterPerClassIntegration
void testGlobalPartitionStatesPartitionsOutOfRange() {
String zoneName = ZONES_CONTAINING_TABLES.stream().findAny().get();
+ int partitionCount = partitionsCount(zoneName);
+
assertThrowsProblem(
() -> client.toBlocking().exchange(
- String.format("%s?partitionIds=0,4,%d&zoneNames=%s",
globalStatePath(), DEFAULT_PARTITION_COUNT, zoneName),
+ String.format("%s?partitionIds=0,4,%d&zoneNames=%s",
globalStatePath(), partitionCount, zoneName),
GlobalZonePartitionStatesResponse.class
),
isProblem().withStatus(BAD_REQUEST)
.withDetail(String.format(
"Partition IDs should be in range [0, %d] for
zone %s, found: %d",
- DEFAULT_PARTITION_COUNT - 1,
+ partitionCount - 1,
zoneName,
- DEFAULT_PARTITION_COUNT
+ partitionCount
))
);
}
@@ -392,15 +408,17 @@ public class ItDisasterRecoveryControllerTest extends
ClusterPerClassIntegration
@Test
void testResetPartitionsPartitionsOutOfRange() {
- MutableHttpRequest<?> post = resetPartitionsRequest(FIRST_ZONE,
QUALIFIED_TABLE_NAME, Set.of(DEFAULT_PARTITION_COUNT));
+ int partitionCount = partitionsCount(FIRST_ZONE);
+
+ MutableHttpRequest<?> post = resetPartitionsRequest(FIRST_ZONE,
QUALIFIED_TABLE_NAME, Set.of(partitionCount));
assertThrowsProblem(
() -> client.toBlocking().exchange(post),
isProblem().withStatus(BAD_REQUEST).withDetail(String.format(
"Partition IDs should be in range [0, %d] for zone %s,
found: %d",
- DEFAULT_PARTITION_COUNT - 1,
+ partitionCount - 1,
FIRST_ZONE,
- DEFAULT_PARTITION_COUNT
+ partitionCount
))
);
}
diff --git a/modules/runner/build.gradle b/modules/runner/build.gradle
index db94bccfa69..5c7a7d3825c 100644
--- a/modules/runner/build.gradle
+++ b/modules/runner/build.gradle
@@ -236,6 +236,7 @@ dependencies {
testFixturesImplementation testFixtures(project(':ignite-catalog'))
testFixturesImplementation testFixtures(project(':ignite-table'))
testFixturesImplementation testFixtures(project(':ignite-storage-api'))
+ testFixturesImplementation
testFixtures(project(':ignite-distribution-zones'))
testFixturesImplementation libs.typesafe.config
testFixturesImplementation libs.awaitility
}
diff --git
a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/ItIgniteNodeRestartTest.java
b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/ItIgniteNodeRestartTest.java
index 60d29f91e79..a30bf71d42e 100644
---
a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/ItIgniteNodeRestartTest.java
+++
b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/ItIgniteNodeRestartTest.java
@@ -89,7 +89,7 @@ import org.apache.ignite.internal.app.NodePropertiesImpl;
import org.apache.ignite.internal.app.ThreadPoolsManager;
import org.apache.ignite.internal.catalog.CatalogManager;
import org.apache.ignite.internal.catalog.CatalogManagerImpl;
-import org.apache.ignite.internal.catalog.PartitionCountProvider;
+import org.apache.ignite.internal.catalog.CatalogTestUtils;
import org.apache.ignite.internal.catalog.commands.AlterZoneCommand;
import org.apache.ignite.internal.catalog.commands.AlterZoneCommandBuilder;
import org.apache.ignite.internal.catalog.commands.ColumnParams;
@@ -614,7 +614,7 @@ public class ItIgniteNodeRestartTest extends
BaseIgniteRestartTest {
clockService,
failureProcessor,
delayDurationMsSupplier,
- PartitionCountProvider.defaultPartitionCountProvider()
+ CatalogTestUtils.defaultPartitionCountCalculator()
);
var registry = new MetaStorageRevisionListenerRegistry(metaStorageMgr);
diff --git
a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/PlatformTestNodeRunner.java
b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/PlatformTestNodeRunner.java
index 52a27b36f59..68254a66639 100644
---
a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/PlatformTestNodeRunner.java
+++
b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/runner/app/PlatformTestNodeRunner.java
@@ -21,6 +21,7 @@ import static java.util.concurrent.CompletableFuture.allOf;
import static java.util.concurrent.CompletableFuture.completedFuture;
import static java.util.stream.Collectors.toList;
import static org.apache.ignite.internal.TestWrappers.unwrapIgniteImpl;
+import static
org.apache.ignite.internal.catalog.commands.CatalogUtils.DEFAULT_PARTITION_COUNT;
import static
org.apache.ignite.internal.catalog.commands.CatalogUtils.MAX_TIME_PRECISION;
import static
org.apache.ignite.internal.distributionzones.DistributionZonesTestUtil.createZone;
import static org.apache.ignite.internal.table.TableTestUtils.createTable;
@@ -81,12 +82,14 @@ import org.apache.ignite.compute.task.MapReduceTask;
import org.apache.ignite.compute.task.TaskExecutionContext;
import org.apache.ignite.deployment.DeploymentUnit;
import org.apache.ignite.internal.app.IgniteImpl;
+import org.apache.ignite.internal.app.IgniteServerImpl;
import org.apache.ignite.internal.binarytuple.BinaryTupleReader;
import org.apache.ignite.internal.catalog.commands.ColumnParams;
import org.apache.ignite.internal.catalog.commands.DefaultValue;
import org.apache.ignite.internal.client.proto.ColumnTypeConverter;
import org.apache.ignite.internal.configuration.ClusterChange;
import org.apache.ignite.internal.configuration.ClusterConfiguration;
+import org.apache.ignite.internal.distributionzones.DistributionZonesTestUtil;
import org.apache.ignite.internal.runner.app.Jobs.JsonMarshaller;
import org.apache.ignite.internal.schema.Column;
import org.apache.ignite.internal.schema.SchemaDescriptor;
@@ -300,6 +303,8 @@ public class PlatformTestNodeRunner {
})
.collect(toList());
+ nodes.forEach(server -> ((IgniteServerImpl)
server).igniteImpl().useStaticPartitionCountCalculator(DEFAULT_PARTITION_COUNT));
+
IgniteServer metaStorageNode = nodes.get(0);
InitParameters initParameters = InitParameters.builder()
@@ -531,6 +536,10 @@ public class PlatformTestNodeRunner {
);
}
+ private static void createDefaultZone(IgniteImpl ignite) {
+ DistributionZonesTestUtil.createDefaultZone(ignite.catalogManager());
+ }
+
/**
* Gets the thin client port.
*
diff --git
a/modules/runner/src/main/java/org/apache/ignite/internal/app/IgniteImpl.java
b/modules/runner/src/main/java/org/apache/ignite/internal/app/IgniteImpl.java
index 1ee7e50daca..6d01270e7ea 100644
---
a/modules/runner/src/main/java/org/apache/ignite/internal/app/IgniteImpl.java
+++
b/modules/runner/src/main/java/org/apache/ignite/internal/app/IgniteImpl.java
@@ -68,7 +68,9 @@ import
org.apache.ignite.configuration.ConfigurationDynamicDefaultsPatcher;
import org.apache.ignite.configuration.KeyIgnorer;
import org.apache.ignite.internal.catalog.CatalogManager;
import org.apache.ignite.internal.catalog.CatalogManagerImpl;
-import org.apache.ignite.internal.catalog.PartitionCountProviderWrapper;
+import
org.apache.ignite.internal.catalog.DataNodesAwarePartitionCountCalculator;
+import org.apache.ignite.internal.catalog.PartitionCountCalculator;
+import org.apache.ignite.internal.catalog.PartitionCountCalculatorWrapper;
import org.apache.ignite.internal.catalog.compaction.CatalogCompactionRunner;
import
org.apache.ignite.internal.catalog.configuration.SchemaSynchronizationConfiguration;
import
org.apache.ignite.internal.catalog.configuration.SchemaSynchronizationExtensionConfiguration;
@@ -262,6 +264,8 @@ import
org.apache.ignite.internal.storage.DataStorageModules;
import
org.apache.ignite.internal.storage.configurations.StorageExtensionConfiguration;
import org.apache.ignite.internal.storage.engine.StorageEngine;
import org.apache.ignite.internal.storage.engine.ThreadAssertingStorageEngine;
+import org.apache.ignite.internal.system.CpuInformationProvider;
+import org.apache.ignite.internal.system.JvmCpuInformationProvider;
import org.apache.ignite.internal.systemview.SystemViewManagerImpl;
import org.apache.ignite.internal.systemview.api.SystemViewManager;
import
org.apache.ignite.internal.table.distributed.PartitionModificationCounterFactory;
@@ -523,7 +527,7 @@ public class IgniteImpl implements Ignite {
private final PartitionModificationCounterFactory
partitionModificationCounterFactory;
- private final PartitionCountProviderWrapper partitionCountProviderWrapper;
+ private final PartitionCountCalculatorWrapper
partitionCountCalculatorWrapper;
/** Future that completes when the node has joined the cluster. */
private final CompletableFuture<Ignite> joinFuture = new
CompletableFuture<>();
@@ -561,6 +565,8 @@ public class IgniteImpl implements Ignite {
nodeProperties = new NodePropertiesImpl(vaultMgr);
+ CpuInformationProvider cpuInformationProvider = new
JvmCpuInformationProvider();
+
ConfigurationModules modules =
ConfigurationModules.create(serviceProviderClassLoader);
ConfigurationTreeGenerator localConfigurationGenerator = new
ConfigurationTreeGenerator(
@@ -876,14 +882,14 @@ public class IgniteImpl implements Ignite {
LongSupplier delayDurationMsSupplier =
delayDurationMsSupplier(schemaSyncConfig);
- partitionCountProviderWrapper = new PartitionCountProviderWrapper();
+ partitionCountCalculatorWrapper = new
PartitionCountCalculatorWrapper();
CatalogManagerImpl catalogManager = new CatalogManagerImpl(
new UpdateLogImpl(metaStorageMgr, failureManager),
clockService,
failureManager,
delayDurationMsSupplier,
- partitionCountProviderWrapper
+ partitionCountCalculatorWrapper
);
ReplicationConfiguration replicationConfig = clusterConfigRegistry
@@ -1026,6 +1032,13 @@ public class IgniteImpl implements Ignite {
lowWatermark
);
+ var dataNodesAwarePartitionCountCalculator = new
DataNodesAwarePartitionCountCalculator(
+ distributionZoneManager::estimatedDataNodesCount,
+ cpuInformationProvider
+ );
+
+
partitionCountCalculatorWrapper.setPartitionCountCalculator(dataNodesAwarePartitionCountCalculator);
+
indexNodeFinishedRwTransactionsChecker = new
IndexNodeFinishedRwTransactionsChecker(
catalogManager,
clusterSvc.messagingService(),
@@ -2233,6 +2246,16 @@ public class IgniteImpl implements Ignite {
return schemaSafeTimeTracker;
}
+ @TestOnly
+ public PartitionCountCalculator partitionCountCalculator() {
+ return partitionCountCalculatorWrapper;
+ }
+
+ @TestOnly
+ public void useStaticPartitionCountCalculator(int partitions) {
+
partitionCountCalculatorWrapper.setPartitionCountCalculator(PartitionCountCalculator.staticPartitionCountCalculator(partitions));
+ }
+
/** Triggers dumping node components state. This method is used for
debugging purposes only. */
@TestOnly
public void dumpClusterState() {
diff --git
a/modules/runner/src/main/java/org/apache/ignite/internal/catalog/DataNodesAwarePartitionCountCalculator.java
b/modules/runner/src/main/java/org/apache/ignite/internal/catalog/DataNodesAwarePartitionCountCalculator.java
new file mode 100644
index 00000000000..d71cdff1514
--- /dev/null
+++
b/modules/runner/src/main/java/org/apache/ignite/internal/catalog/DataNodesAwarePartitionCountCalculator.java
@@ -0,0 +1,66 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.catalog;
+
+import static java.lang.Math.max;
+
+import org.apache.ignite.internal.system.CpuInformationProvider;
+
+/**
+ * Primary to use partition count calculator. It calculates the number of
partitions using the formula:
+ * dataNodesCount * max(cores, 8) * scaleFactor / replicas. Data nodes count
is the estimated number of data nodes for the given
+ * distribution zone. It is assumed that each node has the same number of CPU
cores. Also, there is doubling multiplier to allow the cluster
+ * scale up.
+ */
+public class DataNodesAwarePartitionCountCalculator implements
PartitionCountCalculator {
+ static final int SCALE_FACTOR = 3;
+
+ static final int MINIMUM_CPU_COUNT = 8;
+
+ private final EstimatedDataNodeCountProvider
estimatedDataNodeCountProvider;
+
+ private final CpuInformationProvider cpuInfoProvider;
+
+ /**
+ * Constructor.
+ *
+ * @param estimatedDataNodeCountProvider Provides estimated data nodes
count based on given zone filter and storage profile list.
+ * @param cpuInfoProvider Provides CPU information for local node hardware
cores calculation.
+ */
+ public DataNodesAwarePartitionCountCalculator(
+ EstimatedDataNodeCountProvider estimatedDataNodeCountProvider,
+ CpuInformationProvider cpuInfoProvider
+ ) {
+ this.estimatedDataNodeCountProvider = estimatedDataNodeCountProvider;
+ this.cpuInfoProvider = cpuInfoProvider;
+ }
+
+ @Override
+ public int calculate(PartitionCountCalculationParameters params) {
+ int estimatedDataNodeCount =
estimatedDataNodeCountProvider.estimatedDataNodeCount(
+ params.dataNodesFilter(),
+ params.storageProfiles()
+ );
+ // Estimated count may be 0 if the filter sorts out all cluster nodes.
+ int dataNodeCount = max(1, estimatedDataNodeCount);
+ int cores = max(cpuInfoProvider.availableProcessors(),
MINIMUM_CPU_COUNT);
+ int replicas = params.replicaFactor();
+
+ return dataNodeCount * cores * SCALE_FACTOR / replicas;
+ }
+}
diff --git
a/modules/catalog/src/main/java/org/apache/ignite/internal/catalog/PartitionCountProvider.java
b/modules/runner/src/main/java/org/apache/ignite/internal/catalog/EstimatedDataNodeCountProvider.java
similarity index 70%
rename from
modules/catalog/src/main/java/org/apache/ignite/internal/catalog/PartitionCountProvider.java
rename to
modules/runner/src/main/java/org/apache/ignite/internal/catalog/EstimatedDataNodeCountProvider.java
index 7029cc53987..40120fcf5b8 100644
---
a/modules/catalog/src/main/java/org/apache/ignite/internal/catalog/PartitionCountProvider.java
+++
b/modules/runner/src/main/java/org/apache/ignite/internal/catalog/EstimatedDataNodeCountProvider.java
@@ -17,16 +17,17 @@
package org.apache.ignite.internal.catalog;
-import static
org.apache.ignite.internal.catalog.commands.CatalogUtils.DEFAULT_PARTITION_COUNT;
+import java.util.List;
/**
- * Default partition count provider.
+ * Provides an estimated count of data nodes.
*/
@FunctionalInterface
-public interface PartitionCountProvider {
- int calculate(PartitionCountCalculationParameters params);
-
- static PartitionCountProvider defaultPartitionCountProvider() {
- return params -> DEFAULT_PARTITION_COUNT;
- }
+public interface EstimatedDataNodeCountProvider {
+ /**
+ * Provides an estimated count of data nodes.
+ *
+ * @return Estimated count of data nodes.
+ */
+ int estimatedDataNodeCount(String dataNodesFilter, List<String>
storageProfiles);
}
diff --git
a/modules/runner/src/main/java/org/apache/ignite/internal/catalog/PartitionCountProviderWrapper.java
b/modules/runner/src/main/java/org/apache/ignite/internal/catalog/PartitionCountCalculatorWrapper.java
similarity index 61%
rename from
modules/runner/src/main/java/org/apache/ignite/internal/catalog/PartitionCountProviderWrapper.java
rename to
modules/runner/src/main/java/org/apache/ignite/internal/catalog/PartitionCountCalculatorWrapper.java
index e6eb07bd5e8..32749d0b80e 100644
---
a/modules/runner/src/main/java/org/apache/ignite/internal/catalog/PartitionCountProviderWrapper.java
+++
b/modules/runner/src/main/java/org/apache/ignite/internal/catalog/PartitionCountCalculatorWrapper.java
@@ -17,21 +17,26 @@
package org.apache.ignite.internal.catalog;
+import org.apache.ignite.internal.catalog.commands.CatalogUtils;
+
/**
* Wrapper that allows to apply and use different partition count computing
approaches.
*/
-public final class PartitionCountProviderWrapper implements
PartitionCountProvider {
+public final class PartitionCountCalculatorWrapper implements
PartitionCountCalculator {
/** Wrapped delegate to compute partition count. */
- private volatile PartitionCountProvider delegate;
+ private volatile PartitionCountCalculator delegate;
/** Constructor. */
- public PartitionCountProviderWrapper(PartitionCountProvider delegate) {
+ public PartitionCountCalculatorWrapper(PartitionCountCalculator delegate) {
this.delegate = delegate;
}
- /** Constructor that uses {@link
PartitionCountProvider#defaultPartitionCountProvider}. */
- public PartitionCountProviderWrapper() {
- this(PartitionCountProvider.defaultPartitionCountProvider());
+ /**
+ * Constructor that uses {@link
PartitionCountCalculator#staticPartitionCountCalculator} with
+ * {@link CatalogUtils#DEFAULT_PARTITION_COUNT} as static calculation
result.
+ * */
+ public PartitionCountCalculatorWrapper() {
+
this(PartitionCountCalculator.staticPartitionCountCalculator(CatalogUtils.DEFAULT_PARTITION_COUNT));
}
@Override
@@ -39,7 +44,7 @@ public final class PartitionCountProviderWrapper implements
PartitionCountProvid
return delegate.calculate(parameters);
}
- public void setPartitionCountProvider(PartitionCountProvider newProvider) {
- this.delegate = newProvider;
+ public void setPartitionCountCalculator(PartitionCountCalculator
newCalculator) {
+ this.delegate = newCalculator;
}
}
diff --git
a/modules/sql-engine/src/integrationTest/java/org/apache/ignite/internal/sql/engine/systemviews/AbstractSystemViewTest.java
b/modules/runner/src/main/java/org/apache/ignite/internal/system/JvmCpuInformationProvider.java
similarity index 72%
copy from
modules/sql-engine/src/integrationTest/java/org/apache/ignite/internal/sql/engine/systemviews/AbstractSystemViewTest.java
copy to
modules/runner/src/main/java/org/apache/ignite/internal/system/JvmCpuInformationProvider.java
index 25c717d2a9f..912de39c798 100644
---
a/modules/sql-engine/src/integrationTest/java/org/apache/ignite/internal/sql/engine/systemviews/AbstractSystemViewTest.java
+++
b/modules/runner/src/main/java/org/apache/ignite/internal/system/JvmCpuInformationProvider.java
@@ -15,12 +15,14 @@
* limitations under the License.
*/
-package org.apache.ignite.internal.sql.engine.systemviews;
-
-import org.apache.ignite.internal.sql.engine.BaseSqlMultiStatementTest;
+package org.apache.ignite.internal.system;
/**
- * Base class for SQL system views integration tests.
+ * Provides CPU information based on JVM {@link Runtime} instance.
*/
-public class AbstractSystemViewTest extends BaseSqlMultiStatementTest {
+public class JvmCpuInformationProvider implements CpuInformationProvider {
+ @Override
+ public int availableProcessors() {
+ return Runtime.getRuntime().availableProcessors();
+ }
}
diff --git
a/modules/runner/src/test/java/org/apache/ignite/internal/catalog/DataNodesAwarePartitionCountCalculatorTest.java
b/modules/runner/src/test/java/org/apache/ignite/internal/catalog/DataNodesAwarePartitionCountCalculatorTest.java
new file mode 100644
index 00000000000..eca255e7261
--- /dev/null
+++
b/modules/runner/src/test/java/org/apache/ignite/internal/catalog/DataNodesAwarePartitionCountCalculatorTest.java
@@ -0,0 +1,124 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.catalog;
+
+import static
org.apache.ignite.internal.catalog.DataNodesAwarePartitionCountCalculator.MINIMUM_CPU_COUNT;
+import static
org.apache.ignite.internal.catalog.DataNodesAwarePartitionCountCalculator.SCALE_FACTOR;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.Matchers.greaterThan;
+import static org.hamcrest.Matchers.is;
+import static org.hamcrest.Matchers.lessThan;
+
+import org.apache.ignite.internal.system.CpuInformationProvider;
+import org.apache.ignite.internal.testframework.BaseIgniteAbstractTest;
+import org.junit.jupiter.api.Test;
+
+/**
+ * Unit tests for {@link DataNodesAwarePartitionCountCalculator}.
+ */
+public class DataNodesAwarePartitionCountCalculatorTest extends
BaseIgniteAbstractTest {
+ @Test
+ void cpuCountLessThatMinimumTest() {
+ EstimatedDataNodeCountProvider minPossibleDataNodeCountProvider = (f,
sp) -> 1;
+
+ CpuInformationProvider minPossibleCpuInfoProvider = new
CpuInformationProvider() {
+ @Override
+ public int availableProcessors() {
+ return 1;
+ }
+ };
+
+ int minReplicaCount = 1;
+
+ DataNodesAwarePartitionCountCalculator partitionCalculator = new
DataNodesAwarePartitionCountCalculator(
+ minPossibleDataNodeCountProvider,
+ minPossibleCpuInfoProvider
+ );
+
+ PartitionCountCalculationParameters params =
PartitionCountCalculationParameters.builder()
+ .replicaFactor(minReplicaCount)
+ .build();
+
+ assertThat(partitionCalculator.calculate(params), is(SCALE_FACTOR *
MINIMUM_CPU_COUNT));
+ }
+
+ @Test
+ void maxValuesDoesNotLeadToOverflowTest() {
+ // We may assume that at the time maximum data nodes count doesnt'
exceed 200_000: RIKEN Fugaku cluster consists of 158_976 nodes.
+ int maxDataNodesCount = 200_000;
+ EstimatedDataNodeCountProvider maxPossibleDataNodeCountProvider = (f,
sp) -> maxDataNodesCount;
+
+ // We may assume that at the time maximum CPU count doesn't exceed
1000: SYS-681E-TR server with 8490H may hosts up to 480 CPUs.
+ int maxCpuCount = 1000;
+ CpuInformationProvider maxPossibleCpuInfoProvider = new
CpuInformationProvider() {
+ @Override
+ public int availableProcessors() {
+ return maxCpuCount;
+ }
+ };
+
+ // Replica factor is the divider in the formula.
+ int minReplicaCount = 1;
+
+ DataNodesAwarePartitionCountCalculator partitionCalculator = new
DataNodesAwarePartitionCountCalculator(
+ maxPossibleDataNodeCountProvider,
+ maxPossibleCpuInfoProvider
+ );
+
+ PartitionCountCalculationParameters params =
PartitionCountCalculationParameters.builder()
+ .replicaFactor(minReplicaCount)
+ .build();
+
+ int partitionCount = partitionCalculator.calculate(params);
+
+ assertThat(partitionCount, is(greaterThan(0)));
+ // We check only overflow case, but the value may be more than
CatalogUtils.MAX_PARTITION_COUNT, but it's ok and proper validators
+ // will catch this case during catalog command processing. But we must
not get overflow exception during the calculation.
+ assertThat(partitionCount, is(lessThan(Integer.MAX_VALUE)));
+ }
+
+ @Test
+ void testEstimatedDataNodeCountIsZeroButPartitionCountIsPositiveValue() {
+ EstimatedDataNodeCountProvider zeroEstimatedNodesProvider = (f, sp) ->
0;
+ int cpuCount = 1;
+
+ CpuInformationProvider cpuInfoProvider = new CpuInformationProvider() {
+ @Override
+ public int availableProcessors() {
+ return cpuCount;
+ }
+ };
+
+ // Replica factor is the divider in the formula.
+ int minReplicaCount = 1;
+
+ DataNodesAwarePartitionCountCalculator partitionCalculator = new
DataNodesAwarePartitionCountCalculator(
+ zeroEstimatedNodesProvider,
+ cpuInfoProvider
+ );
+
+ PartitionCountCalculationParameters params =
PartitionCountCalculationParameters.builder()
+ .replicaFactor(minReplicaCount)
+ .build();
+
+ int partitionCount = partitionCalculator.calculate(params);
+
+ assertThat(partitionCount, is(greaterThan(0)));
+ assertThat(partitionCount, is(MINIMUM_CPU_COUNT * SCALE_FACTOR));
+ }
+}
diff --git
a/modules/runner/src/test/java/org/apache/ignite/internal/catalog/PartitionCountProviderWrapperTest.java
b/modules/runner/src/test/java/org/apache/ignite/internal/catalog/PartitionCountCalculatorWrapperTest.java
similarity index 72%
rename from
modules/runner/src/test/java/org/apache/ignite/internal/catalog/PartitionCountProviderWrapperTest.java
rename to
modules/runner/src/test/java/org/apache/ignite/internal/catalog/PartitionCountCalculatorWrapperTest.java
index 782e337a0a3..72685eb916d 100644
---
a/modules/runner/src/test/java/org/apache/ignite/internal/catalog/PartitionCountProviderWrapperTest.java
+++
b/modules/runner/src/test/java/org/apache/ignite/internal/catalog/PartitionCountCalculatorWrapperTest.java
@@ -25,14 +25,14 @@ import java.util.concurrent.atomic.AtomicInteger;
import org.apache.ignite.internal.testframework.BaseIgniteAbstractTest;
import org.junit.jupiter.api.Test;
-/** Unit tests for {@link PartitionCountProviderWrapper}. */
-public class PartitionCountProviderWrapperTest extends BaseIgniteAbstractTest {
+/** Unit tests for {@link PartitionCountCalculatorWrapper}. */
+public class PartitionCountCalculatorWrapperTest extends
BaseIgniteAbstractTest {
/**
- * Tests that the default constructor uses the default partition count
provider.
+ * Tests that the default constructor uses the default partition count
calculator.
*/
@Test
void testDefaultConstructor() {
- PartitionCountProviderWrapper wrapper = new
PartitionCountProviderWrapper();
+ PartitionCountCalculatorWrapper wrapper = new
PartitionCountCalculatorWrapper();
PartitionCountCalculationParameters params =
PartitionCountCalculationParameters.builder().build();
int result = wrapper.calculate(params);
@@ -41,21 +41,21 @@ public class PartitionCountProviderWrapperTest extends
BaseIgniteAbstractTest {
}
/**
- * Tests that the constructor with custom provider correctly wraps it.
+ * Tests that the constructor with custom partition calculator correctly
wraps it.
*/
@Test
- void testConstructorWithCustomProvider() {
+ void testConstructorWithCustomPartitionCalculator() {
int expectedPartitionCount = 42;
AtomicInteger callCount = new AtomicInteger(0);
- PartitionCountProvider customProvider = params -> {
+ PartitionCountCalculator customCalculator = params -> {
callCount.incrementAndGet();
return expectedPartitionCount;
};
- PartitionCountProviderWrapper wrapper = new
PartitionCountProviderWrapper(customProvider);
+ PartitionCountCalculatorWrapper wrapper = new
PartitionCountCalculatorWrapper(customCalculator);
PartitionCountCalculationParameters calculationParameters =
PartitionCountCalculationParameters.builder().build();
int result = wrapper.calculate(calculationParameters);
@@ -67,38 +67,38 @@ public class PartitionCountProviderWrapperTest extends
BaseIgniteAbstractTest {
}
/**
- * Tests that calculate uses new provider after it has been changed
several times.
+ * Tests that calculate uses new partition calculator after it has been
changed several times.
*/
@Test
- void testCalculateAfterProviderChange() {
+ void testCalculateAfterPartitionCalculatorChange() {
- PartitionCountProviderWrapper wrapper = new
PartitionCountProviderWrapper();
+ PartitionCountCalculatorWrapper wrapper = new
PartitionCountCalculatorWrapper();
PartitionCountCalculationParameters calculationParameters =
PartitionCountCalculationParameters.builder().build();
assertEquals(DEFAULT_PARTITION_COUNT,
wrapper.calculate(calculationParameters));
int expectedFirstlyChangedPartitionCount = 10;
- PartitionCountProvider firstChangeProvider = params ->
expectedFirstlyChangedPartitionCount;
- wrapper.setPartitionCountProvider(firstChangeProvider);
+ PartitionCountCalculator firstChangeCalculator = params ->
expectedFirstlyChangedPartitionCount;
+ wrapper.setPartitionCountCalculator(firstChangeCalculator);
assertEquals(expectedFirstlyChangedPartitionCount,
wrapper.calculate(calculationParameters));
assertNotEquals(expectedFirstlyChangedPartitionCount,
DEFAULT_PARTITION_COUNT);
int expectedLastlyChangedPartitionCount = 20;
- PartitionCountProvider lastChangeProvider = params ->
expectedLastlyChangedPartitionCount;
- wrapper.setPartitionCountProvider(lastChangeProvider);
+ PartitionCountCalculator lastChangeCalculator = params ->
expectedLastlyChangedPartitionCount;
+ wrapper.setPartitionCountCalculator(lastChangeCalculator);
assertEquals(expectedLastlyChangedPartitionCount,
wrapper.calculate(calculationParameters));
assertNotEquals(expectedFirstlyChangedPartitionCount,
DEFAULT_PARTITION_COUNT);
assertNotEquals(expectedFirstlyChangedPartitionCount,
expectedLastlyChangedPartitionCount);
}
/**
- * Tests that calculate works correctly with different parameter
combinations.
+ * Tests that calculate function works correctly with different parameter
combinations.
*/
@Test
void testCalculateWithDifferentParameters() {
- PartitionCountProvider doubleProvider = params ->
params.replicaFactor() * 2;
+ PartitionCountCalculator doubleCalculator = params ->
params.replicaFactor() * 2;
- PartitionCountProviderWrapper wrapper = new
PartitionCountProviderWrapper(doubleProvider);
+ PartitionCountCalculatorWrapper wrapper = new
PartitionCountCalculatorWrapper(doubleCalculator);
PartitionCountCalculationParameters params1 =
PartitionCountCalculationParameters.builder()
.replicaFactor(2)
diff --git
a/modules/runner/src/test/java/org/apache/ignite/internal/system/JvmCpuInformationProviderTest.java
b/modules/runner/src/test/java/org/apache/ignite/internal/system/JvmCpuInformationProviderTest.java
new file mode 100644
index 00000000000..d6b67a7e937
--- /dev/null
+++
b/modules/runner/src/test/java/org/apache/ignite/internal/system/JvmCpuInformationProviderTest.java
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.system;
+
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.Matchers.greaterThanOrEqualTo;
+import static org.hamcrest.Matchers.is;
+
+import org.apache.ignite.internal.testframework.BaseIgniteAbstractTest;
+import org.junit.jupiter.api.Test;
+
+/**
+ * Unit tests for {@link JvmCpuInformationProvider}.
+ */
+public class JvmCpuInformationProviderTest extends BaseIgniteAbstractTest {
+ @Test
+ void availableProcessorsIsPositiveNumberTest() {
+ JvmCpuInformationProvider cpuInformationProvider = new
JvmCpuInformationProvider();
+
+ assertThat(cpuInformationProvider.availableProcessors(),
is(greaterThanOrEqualTo(1)));
+ }
+
+ @Test
+ void availableProcessorsAreCalculatedTheSameAsJvmRuntime() {
+ JvmCpuInformationProvider cpuInformationProvider = new
JvmCpuInformationProvider();
+
+ assertThat(cpuInformationProvider.availableProcessors(),
is(Runtime.getRuntime().availableProcessors()));
+ }
+}
diff --git
a/modules/runner/src/testFixtures/java/org/apache/ignite/internal/ClusterPerClassIntegrationTest.java
b/modules/runner/src/testFixtures/java/org/apache/ignite/internal/ClusterPerClassIntegrationTest.java
index 9be112ea36e..990fd4c6e30 100644
---
a/modules/runner/src/testFixtures/java/org/apache/ignite/internal/ClusterPerClassIntegrationTest.java
+++
b/modules/runner/src/testFixtures/java/org/apache/ignite/internal/ClusterPerClassIntegrationTest.java
@@ -19,7 +19,6 @@ package org.apache.ignite.internal;
import static
org.apache.ignite.internal.ConfigTemplates.NODE_BOOTSTRAP_CFG_TEMPLATE;
import static org.apache.ignite.internal.TestWrappers.unwrapIgniteImpl;
-import static
org.apache.ignite.internal.catalog.CatalogService.DEFAULT_STORAGE_PROFILE;
import static
org.apache.ignite.internal.catalog.descriptors.CatalogIndexStatus.AVAILABLE;
import static org.apache.ignite.internal.lang.IgniteStringFormatter.format;
import static
org.apache.ignite.internal.testframework.IgniteTestUtils.getAllResultSet;
@@ -27,6 +26,11 @@ import static
org.apache.ignite.internal.testframework.IgniteTestUtils.waitForCo
import static
org.apache.ignite.internal.testframework.matchers.CompletableFutureMatcher.willCompleteSuccessfully;
import static org.apache.ignite.lang.util.IgniteNameUtils.quoteIfNeeded;
import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.Matchers.empty;
+import static org.hamcrest.Matchers.is;
+import static org.hamcrest.Matchers.not;
+import static org.hamcrest.Matchers.notNullValue;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
import static org.junit.jupiter.api.Assertions.assertTrue;
import java.nio.file.Path;
@@ -44,13 +48,16 @@ import java.util.stream.Collectors;
import java.util.stream.Stream;
import org.apache.ignite.Ignite;
import org.apache.ignite.InitParametersBuilder;
+import org.apache.ignite.catalog.definitions.ZoneDefinition;
import org.apache.ignite.internal.app.IgniteImpl;
import org.apache.ignite.internal.catalog.Catalog;
import org.apache.ignite.internal.catalog.CatalogManager;
+import org.apache.ignite.internal.catalog.CatalogService;
import org.apache.ignite.internal.catalog.commands.CatalogUtils;
import org.apache.ignite.internal.catalog.descriptors.CatalogIndexDescriptor;
import org.apache.ignite.internal.catalog.descriptors.CatalogSchemaDescriptor;
import org.apache.ignite.internal.catalog.descriptors.CatalogZoneDescriptor;
+import org.apache.ignite.internal.distributionzones.DistributionZonesTestUtil;
import org.apache.ignite.internal.hlc.HybridClock;
import org.apache.ignite.internal.lang.IgniteBiTuple;
import org.apache.ignite.internal.sql.SqlCommon;
@@ -115,8 +122,14 @@ public abstract class ClusterPerClassIntegrationTest
extends BaseIgniteAbstractT
CLUSTER = new Cluster(clusterConfiguration.build());
- if (initialNodes() > 0 && needInitializeCluster()) {
- CLUSTER.startAndInit(testInfo, initialNodes(),
cmgMetastoreNodes(), this::configureInitParameters);
+ if (!shouldStartAndInitializeCluster()) {
+ return;
+ }
+
+ CLUSTER.startAndInit(testInfo, initialNodes(), cmgMetastoreNodes(),
this::configureInitParameters);
+
+ if (shouldCreateDefaultZone()) {
+ createDefaultZone();
}
}
@@ -158,6 +171,21 @@ public abstract class ClusterPerClassIntegrationTest
extends BaseIgniteAbstractT
return NODE_BOOTSTRAP_CFG_TEMPLATE;
}
+ private boolean shouldStartAndInitializeCluster() {
+ return initialNodes() > 0 && needInitializeCluster();
+ }
+
+ protected boolean shouldCreateDefaultZone() {
+ return true;
+ }
+
+ private static void createDefaultZone() {
+ assertThat(CLUSTER, is(notNullValue()));
+ assertThat(CLUSTER.nodes(), is(not(empty())));
+
+
DistributionZonesTestUtil.createDefaultZone(igniteImpl(0).catalogManager());
+ }
+
/**
* After all.
*/
@@ -348,7 +376,7 @@ public abstract class ClusterPerClassIntegrationTest
extends BaseIgniteAbstractT
* @param partitions Partitions count.
*/
protected static Table createZoneAndTable(String zoneName, String
tableName, int replicas, int partitions) {
- createZoneOnlyIfNotExists(zoneName, replicas, partitions,
DEFAULT_STORAGE_PROFILE);
+ createZoneOnlyIfNotExists(zoneName, replicas, partitions,
CatalogService.DEFAULT_STORAGE_PROFILE);
return createTableOnly(tableName, zoneName);
}
@@ -704,6 +732,20 @@ public abstract class ClusterPerClassIntegrationTest
extends BaseIgniteAbstractT
return unwrapIgniteImpl(node).node().toPublicNode();
}
+ /**
+ * Returns partition count by given zone name.
+ *
+ * @param zoneName Zone to get partition count for.
+ * @return Partition count for given zone name assumed the zone exists.
+ */
+ protected static int partitionsCount(String zoneName) {
+ ZoneDefinition zoneDescriptor =
CLUSTER.aliveNode().catalog().zoneDefinition(zoneName);
+
+ assertNotNull(zoneDescriptor);
+
+ return zoneDescriptor.partitions();
+ }
+
/** Ad-hoc registered extension for dumping cluster state in case of test
failure. */
@RegisterExtension
static ClusterStateDumpingExtension testFailureHook = new
ClusterStateDumpingExtension();
diff --git
a/modules/runner/src/testFixtures/java/org/apache/ignite/internal/ClusterPerTestIntegrationTest.java
b/modules/runner/src/testFixtures/java/org/apache/ignite/internal/ClusterPerTestIntegrationTest.java
index 23ef2a861b3..322f9025a34 100644
---
a/modules/runner/src/testFixtures/java/org/apache/ignite/internal/ClusterPerTestIntegrationTest.java
+++
b/modules/runner/src/testFixtures/java/org/apache/ignite/internal/ClusterPerTestIntegrationTest.java
@@ -20,6 +20,11 @@ package org.apache.ignite.internal;
import static
org.apache.ignite.internal.ConfigTemplates.NODE_BOOTSTRAP_CFG_TEMPLATE;
import static org.apache.ignite.internal.TestWrappers.unwrapIgniteImpl;
import static
org.apache.ignite.internal.testframework.IgniteTestUtils.getAllResultSet;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.Matchers.empty;
+import static org.hamcrest.Matchers.is;
+import static org.hamcrest.Matchers.not;
+import static org.hamcrest.Matchers.notNullValue;
import java.nio.file.Path;
import java.util.List;
@@ -30,6 +35,7 @@ import java.util.stream.Stream;
import org.apache.ignite.Ignite;
import org.apache.ignite.InitParametersBuilder;
import org.apache.ignite.internal.app.IgniteImpl;
+import org.apache.ignite.internal.distributionzones.DistributionZonesTestUtil;
import org.apache.ignite.internal.logger.IgniteLogger;
import org.apache.ignite.internal.logger.Loggers;
import org.apache.ignite.internal.network.InternalClusterNode;
@@ -81,8 +87,14 @@ public abstract class ClusterPerTestIntegrationTest extends
BaseIgniteAbstractTe
cluster = new Cluster(clusterConfiguration.build());
- if (initialNodes() > 0) {
- cluster.startAndInit(testInfo, initialNodes(),
cmgMetastoreNodes(), this::customizeInitParameters);
+ if (!shouldStartAndInitializeCluster()) {
+ return;
+ }
+
+ cluster.startAndInit(testInfo, initialNodes(), cmgMetastoreNodes(),
this::customizeInitParameters);
+
+ if (shouldCreateDefaultZone()) {
+ createDefaultZone();
}
}
@@ -117,6 +129,21 @@ public abstract class ClusterPerTestIntegrationTest
extends BaseIgniteAbstractTe
// No-op.
}
+ private boolean shouldStartAndInitializeCluster() {
+ return initialNodes() > 0;
+ }
+
+ protected boolean shouldCreateDefaultZone() {
+ return true;
+ }
+
+ private void createDefaultZone() {
+ assertThat(cluster, is(notNullValue()));
+ assertThat(cluster.nodes(), is(not(empty())));
+
+
DistributionZonesTestUtil.createDefaultZone(igniteImpl(0).catalogManager());
+ }
+
/**
* Returns node bootstrap config template.
*
diff --git
a/modules/schema-sync/src/integrationTest/java/org/apache/ignite/internal/schemasync/ItSchemaForwardCompatibilityConsistencyTest.java
b/modules/schema-sync/src/integrationTest/java/org/apache/ignite/internal/schemasync/ItSchemaForwardCompatibilityConsistencyTest.java
index 5e61976adfa..0f2bb0ad762 100644
---
a/modules/schema-sync/src/integrationTest/java/org/apache/ignite/internal/schemasync/ItSchemaForwardCompatibilityConsistencyTest.java
+++
b/modules/schema-sync/src/integrationTest/java/org/apache/ignite/internal/schemasync/ItSchemaForwardCompatibilityConsistencyTest.java
@@ -31,6 +31,7 @@ import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.hasSize;
import static org.hamcrest.Matchers.is;
import static org.hamcrest.Matchers.notNullValue;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
import static org.junit.jupiter.api.Assertions.assertTrue;
import java.util.ArrayList;
@@ -42,10 +43,14 @@ import java.util.concurrent.ExecutorService;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
import org.apache.ignite.Ignite;
+import org.apache.ignite.catalog.annotations.Table;
import org.apache.ignite.internal.ClusterPerTestIntegrationTest;
import org.apache.ignite.internal.app.IgniteImpl;
import org.apache.ignite.internal.catalog.Catalog;
+import org.apache.ignite.internal.catalog.CatalogManager;
+import org.apache.ignite.internal.catalog.commands.CatalogUtils;
import org.apache.ignite.internal.catalog.descriptors.CatalogTableDescriptor;
+import org.apache.ignite.internal.catalog.descriptors.CatalogZoneDescriptor;
import org.apache.ignite.internal.hlc.HybridTimestamp;
import org.apache.ignite.internal.schema.BinaryRow;
import org.apache.ignite.internal.storage.MvPartitionStorage;
@@ -71,7 +76,7 @@ import org.junit.jupiter.params.provider.EnumSource;
abstract class ItSchemaForwardCompatibilityConsistencyTest extends
ClusterPerTestIntegrationTest {
private static final String ZONE_NAME = "TEST_ZONE";
- private static final int PARTITION_COUNT = 25;
+ private static final int PARTITION_COUNT =
CatalogUtils.DEFAULT_PARTITION_COUNT;
private Ignite node0;
@@ -81,6 +86,11 @@ abstract class ItSchemaForwardCompatibilityConsistencyTest
extends ClusterPerTes
@Override
protected abstract int initialNodes();
+ @Override
+ protected boolean shouldCreateDefaultZone() {
+ return false;
+ }
+
@BeforeEach
void prepare() {
node0 = cluster.node(0);
@@ -193,7 +203,11 @@ abstract class ItSchemaForwardCompatibilityConsistencyTest
extends ClusterPerTes
IgniteImpl ignite = unwrapIgniteImpl(node);
TableImpl table =
unwrapTableImpl(requireNonNull(ignite.distributedTableManager().cachedTable(tableName)));
- for (int partitionIndex = 0; partitionIndex < PARTITION_COUNT;
partitionIndex++) {
+ int partitionCount = partitions(ignite.catalogManager(), tableName);
+
+ assertThat(partitionCount, is(PARTITION_COUNT));
+
+ for (int partitionIndex = 0; partitionIndex < partitionCount;
partitionIndex++) {
collectRowsFromPartition(table, partitionIndex, readResults);
}
@@ -208,6 +222,20 @@ abstract class ItSchemaForwardCompatibilityConsistencyTest
extends ClusterPerTes
}
}
+ private static int partitions(CatalogManager catalogManager, String
tableName) {
+ Catalog catalog = catalogManager.latestCatalog();
+
+ CatalogTableDescriptor tableDescriptor =
catalog.table(Table.DEFAULT_SCHEMA, tableName);
+
+ assertNotNull(tableDescriptor);
+
+ CatalogZoneDescriptor zoneDescriptor =
catalog.zone(tableDescriptor.zoneId());
+
+ assertNotNull(zoneDescriptor);
+
+ return zoneDescriptor.partitions();
+ }
+
private static void collectRowsFromPartition(TableImpl table, int
partitionIndex, List<ReadResult> readResults) {
MvPartitionStorage partitionStorage =
table.internalTable().storage().getMvPartition(partitionIndex);
diff --git a/modules/sql-engine/build.gradle b/modules/sql-engine/build.gradle
index afbfef97870..e495f7d5419 100644
--- a/modules/sql-engine/build.gradle
+++ b/modules/sql-engine/build.gradle
@@ -146,6 +146,8 @@ dependencies {
integrationTestImplementation testFixtures(project(':ignite-sql-engine'))
integrationTestImplementation testFixtures(project(':ignite-table'))
integrationTestImplementation testFixtures(project(':ignite-runner'))
+ integrationTestImplementation
testFixtures(project(':ignite-distribution-zones'))
+
testFixturesImplementation project(':ignite-core')
testFixturesImplementation project(':ignite-api')
diff --git
a/modules/sql-engine/src/integrationTest/java/org/apache/ignite/internal/sql/api/ItSqlCreateZoneTest.java
b/modules/sql-engine/src/integrationTest/java/org/apache/ignite/internal/sql/api/ItSqlCreateZoneTest.java
index 87d0a134deb..04d14234df5 100644
---
a/modules/sql-engine/src/integrationTest/java/org/apache/ignite/internal/sql/api/ItSqlCreateZoneTest.java
+++
b/modules/sql-engine/src/integrationTest/java/org/apache/ignite/internal/sql/api/ItSqlCreateZoneTest.java
@@ -75,6 +75,11 @@ class ItSqlCreateZoneTest extends
ClusterPerTestIntegrationTest {
return 1;
}
+ @Override
+ protected boolean shouldCreateDefaultZone() {
+ return false;
+ }
+
@Test
void testCreateZoneSucceedWithCorrectStorageProfileOnSameNode() {
assertDoesNotThrow(() -> createZoneQuery(0, DEFAULT_STORAGE_PROFILE));
diff --git
a/modules/sql-engine/src/integrationTest/java/org/apache/ignite/internal/sql/engine/systemviews/AbstractSystemViewTest.java
b/modules/sql-engine/src/integrationTest/java/org/apache/ignite/internal/sql/engine/systemviews/AbstractSystemViewTest.java
index 25c717d2a9f..82d9bb1f870 100644
---
a/modules/sql-engine/src/integrationTest/java/org/apache/ignite/internal/sql/engine/systemviews/AbstractSystemViewTest.java
+++
b/modules/sql-engine/src/integrationTest/java/org/apache/ignite/internal/sql/engine/systemviews/AbstractSystemViewTest.java
@@ -23,4 +23,8 @@ import
org.apache.ignite.internal.sql.engine.BaseSqlMultiStatementTest;
* Base class for SQL system views integration tests.
*/
public class AbstractSystemViewTest extends BaseSqlMultiStatementTest {
+ @Override
+ protected boolean shouldCreateDefaultZone() {
+ return false;
+ }
}
diff --git
a/modules/sql-engine/src/integrationTest/java/org/apache/ignite/internal/sql/engine/systemviews/ItZonesSystemViewTest.java
b/modules/sql-engine/src/integrationTest/java/org/apache/ignite/internal/sql/engine/systemviews/ItZonesSystemViewTest.java
index 1934ad13533..fc76b1842a2 100644
---
a/modules/sql-engine/src/integrationTest/java/org/apache/ignite/internal/sql/engine/systemviews/ItZonesSystemViewTest.java
+++
b/modules/sql-engine/src/integrationTest/java/org/apache/ignite/internal/sql/engine/systemviews/ItZonesSystemViewTest.java
@@ -28,10 +28,13 @@ import static
org.apache.ignite.internal.catalog.commands.CatalogUtils.IMMEDIATE
import static
org.apache.ignite.internal.catalog.commands.CatalogUtils.INFINITE_TIMER_VALUE;
import static
org.apache.ignite.internal.testframework.IgniteTestUtils.assertThrows;
+import java.util.List;
import java.util.Objects;
import org.apache.ignite.internal.app.IgniteImpl;
import org.apache.ignite.internal.catalog.Catalog;
import org.apache.ignite.internal.catalog.CatalogManager;
+import org.apache.ignite.internal.catalog.PartitionCountCalculationParameters;
+import org.apache.ignite.internal.catalog.PartitionCountCalculator;
import org.apache.ignite.internal.catalog.descriptors.ConsistencyMode;
import org.apache.ignite.internal.sql.engine.util.MetadataMatcher;
import org.apache.ignite.sql.ColumnType;
@@ -62,11 +65,19 @@ public class ItZonesSystemViewTest extends
AbstractSystemViewTest {
catalogManager.catalog(catalogManager.activeCatalogVersion(node.clock().nowLong()))
);
+ PartitionCountCalculator partitionCountCalculator =
node.partitionCountCalculator();
+ PartitionCountCalculationParameters
partitionCountCalculationParameters =
PartitionCountCalculationParameters.builder()
+ .replicaFactor(DEFAULT_REPLICA_COUNT)
+ .dataNodesFilter(DEFAULT_FILTER)
+ .storageProfiles(List.of(DEFAULT_STORAGE_PROFILE))
+ .build();
+ int defaultZoneExpectedPartitionCount =
partitionCountCalculator.calculate(partitionCountCalculationParameters);
+
assertQuery("SELECT ZONE_NAME, ZONE_PARTITIONS, ZONE_REPLICAS,
ZONE_QUORUM_SIZE, DATA_NODES_AUTO_ADJUST_SCALE_UP,"
+ " DATA_NODES_AUTO_ADJUST_SCALE_DOWN, DATA_NODES_FILTER,
IS_DEFAULT_ZONE, ZONE_CONSISTENCY_MODE FROM SYSTEM.ZONES")
.returns(
catalog.defaultZone().name(),
- DEFAULT_PARTITION_COUNT,
+ defaultZoneExpectedPartitionCount,
DEFAULT_REPLICA_COUNT,
DEFAULT_ZONE_QUORUM_SIZE,
IMMEDIATE_TIMER_VALUE,
diff --git
a/modules/sql-engine/src/integrationTest/java/org/apache/ignite/internal/sql/sqllogic/ItSqlLogicTest.java
b/modules/sql-engine/src/integrationTest/java/org/apache/ignite/internal/sql/sqllogic/ItSqlLogicTest.java
index 9d0fcf2cfbc..2644b79be24 100644
---
a/modules/sql-engine/src/integrationTest/java/org/apache/ignite/internal/sql/sqllogic/ItSqlLogicTest.java
+++
b/modules/sql-engine/src/integrationTest/java/org/apache/ignite/internal/sql/sqllogic/ItSqlLogicTest.java
@@ -21,6 +21,9 @@ import static java.util.stream.Collectors.toList;
import static org.apache.ignite.internal.TestWrappers.unwrapIgniteImpl;
import static
org.apache.ignite.internal.testframework.matchers.CompletableFutureMatcher.willCompleteSuccessfully;
import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.Matchers.empty;
+import static org.hamcrest.Matchers.is;
+import static org.hamcrest.Matchers.not;
import static org.junit.jupiter.api.Assertions.assertTimeoutPreemptively;
import static org.junit.jupiter.api.Assertions.assertTrue;
@@ -45,6 +48,7 @@ import org.apache.ignite.Ignite;
import org.apache.ignite.IgniteServer;
import org.apache.ignite.InitParameters;
import org.apache.ignite.internal.app.IgniteImpl;
+import org.apache.ignite.internal.distributionzones.DistributionZonesTestUtil;
import org.apache.ignite.internal.lang.IgniteStringFormatter;
import org.apache.ignite.internal.lang.IgniteSystemProperties;
import org.apache.ignite.internal.logger.IgniteLogger;
@@ -382,6 +386,14 @@ public class ItSqlLogicTest extends BaseIgniteAbstractTest
{
enableMetrics(ignite, enabledMetrics);
}
+
+ createDefaultZone();
+ }
+
+ private static void createDefaultZone() {
+ assertThat(CLUSTER_NODES, is(not(empty())));
+
+
DistributionZonesTestUtil.createDefaultZone(unwrapIgniteImpl(CLUSTER_NODES.get(0)).catalogManager());
}
/** Disables all metrics except provided ones. */
diff --git
a/modules/table/src/integrationTest/java/org/apache/ignite/internal/table/distributed/disaster/ItHighAvailablePartitionsRecoveryByFilterUpdateTest.java
b/modules/table/src/integrationTest/java/org/apache/ignite/internal/table/distributed/disaster/ItHighAvailablePartitionsRecoveryByFilterUpdateTest.java
index 94eb712c079..7af3be4746a 100644
---
a/modules/table/src/integrationTest/java/org/apache/ignite/internal/table/distributed/disaster/ItHighAvailablePartitionsRecoveryByFilterUpdateTest.java
+++
b/modules/table/src/integrationTest/java/org/apache/ignite/internal/table/distributed/disaster/ItHighAvailablePartitionsRecoveryByFilterUpdateTest.java
@@ -196,6 +196,7 @@ public class
ItHighAvailablePartitionsRecoveryByFilterUpdateTest extends Abstrac
* @throws Exception If failed.
*/
@Test
+ @Disabled("https://issues.apache.org/jira/browse/IGNITE-28013")
void testSeveralHaResetsAndSomeNodeRestart() throws Exception {
for (int i = 1; i < 8; i++) {
startNode(i, CUSTOM_NODES_CONFIG);
diff --git
a/modules/transactions/src/integrationTest/java/org/apache/ignite/internal/disaster/ItDisasterRecoveryManagerTest.java
b/modules/transactions/src/integrationTest/java/org/apache/ignite/internal/disaster/ItDisasterRecoveryManagerTest.java
index 1d79d93d304..94d47ad7aa8 100644
---
a/modules/transactions/src/integrationTest/java/org/apache/ignite/internal/disaster/ItDisasterRecoveryManagerTest.java
+++
b/modules/transactions/src/integrationTest/java/org/apache/ignite/internal/disaster/ItDisasterRecoveryManagerTest.java
@@ -219,8 +219,9 @@ public class ItDisasterRecoveryManagerTest extends
ClusterPerTestIntegrationTest
assertThat(localStateTableFuture, willCompleteSuccessfully());
Map<ZonePartitionId, LocalPartitionStateByNode> localState =
localStateTableFuture.get();
- // A custom zone, which was created in `BeforeEach` with 2 partitions
due to this test's `ZoneParam` annotation's parameter.
- assertThat(localState, aMapWithSize(2));
+ // A default zone and a custom zone, which was created in `BeforeEach`
with 2 partitions due to this test's `ZoneParam` annotation's
+ // parameter. 27 partitions = CatalogUtils.DEFAULT_PARTITION_COUNT
(=25) + 2.
+ assertThat(localState, aMapWithSize(27));
int zoneId = zoneId(node);
@@ -253,8 +254,9 @@ public class ItDisasterRecoveryManagerTest extends
ClusterPerTestIntegrationTest
assertThat(globalStatesFuture, willCompleteSuccessfully());
Map<ZonePartitionId, GlobalPartitionState> globalState =
globalStatesFuture.get();
- // A custom zone, which was created in `BeforeEach` with 2 partitions
due to this test's `ZoneParam` annotation's parameter.
- assertThat(globalState, aMapWithSize(2));
+ // A default zone and a custom zone, which was created in `BeforeEach`
with 2 partitions due to this test's `ZoneParam` annotation's
+ // parameter. 27 partitions = CatalogUtils.DEFAULT_PARTITION_COUNT
(=25) + 2.
+ assertThat(globalState, aMapWithSize(27));
int zoneId = zoneId(node);