[ignite-3] branch main updated: IGNITE-19219 Implement primary keys query in ODBC (#2699)

2023-10-16 Thread isapego
This is an automated email from the ASF dual-hosted git repository.

isapego 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 b49dd9611c IGNITE-19219  Implement primary keys query in ODBC (#2699)
b49dd9611c is described below

commit b49dd9611c2dc5a732d41a7f7cc725339ecb27b9
Author: Igor Sapego 
AuthorDate: Tue Oct 17 10:30:38 2023 +0400

IGNITE-19219  Implement primary keys query in ODBC (#2699)
---
 modules/platforms/cpp/ignite/odbc/CMakeLists.txt   |   1 +
 .../ignite/odbc/query/column_metadata_query.cpp|   6 +-
 ...e_metadata_query.cpp => primary_keys_query.cpp} | 204 +++--
 .../cpp/ignite/odbc/query/primary_keys_query.h | 137 ++
 modules/platforms/cpp/ignite/odbc/query/query.h|   3 +
 .../cpp/ignite/odbc/query/table_metadata_query.cpp |   6 +-
 .../platforms/cpp/ignite/odbc/sql_statement.cpp|   9 +-
 modules/platforms/cpp/ignite/odbc/sql_statement.h  |   2 +-
 .../cpp/ignite/protocol/client_operation.h |   3 +
 .../cpp/tests/odbc-test/api_robustness_test.cpp| 146 ---
 .../cpp/tests/odbc-test/meta_queries_test.cpp  |  75 
 11 files changed, 413 insertions(+), 179 deletions(-)

diff --git a/modules/platforms/cpp/ignite/odbc/CMakeLists.txt 
b/modules/platforms/cpp/ignite/odbc/CMakeLists.txt
index 3fea05ef88..581b3b89af 100644
--- a/modules/platforms/cpp/ignite/odbc/CMakeLists.txt
+++ b/modules/platforms/cpp/ignite/odbc/CMakeLists.txt
@@ -38,6 +38,7 @@ set(SOURCES
 query/column_metadata_query.cpp
 query/data_query.cpp
 query/foreign_keys_query.cpp
+query/primary_keys_query.cpp
 query/table_metadata_query.cpp
 query/type_info_query.cpp
 odbc.cpp
diff --git a/modules/platforms/cpp/ignite/odbc/query/column_metadata_query.cpp 
b/modules/platforms/cpp/ignite/odbc/query/column_metadata_query.cpp
index fb31c62911..9edab15e3d 100644
--- a/modules/platforms/cpp/ignite/odbc/query/column_metadata_query.cpp
+++ b/modules/platforms/cpp/ignite/odbc/query/column_metadata_query.cpp
@@ -167,10 +167,8 @@ sql_result 
column_metadata_query::fetch_next_row(column_binding_map &column_bind
 if (m_cursor == m_meta.end())
 return sql_result::AI_NO_DATA;
 
-column_binding_map::iterator it;
-
-for (it = column_bindings.begin(); it != column_bindings.end(); ++it)
-get_column(it->first, it->second);
+for (auto &binding : column_bindings)
+get_column(binding.first, binding.second);
 
 return sql_result::AI_SUCCESS;
 }
diff --git a/modules/platforms/cpp/ignite/odbc/query/table_metadata_query.cpp 
b/modules/platforms/cpp/ignite/odbc/query/primary_keys_query.cpp
similarity index 53%
copy from modules/platforms/cpp/ignite/odbc/query/table_metadata_query.cpp
copy to modules/platforms/cpp/ignite/odbc/query/primary_keys_query.cpp
index 710ca8eb28..6d81f467ac 100644
--- a/modules/platforms/cpp/ignite/odbc/query/table_metadata_query.cpp
+++ b/modules/platforms/cpp/ignite/odbc/query/primary_keys_query.cpp
@@ -18,10 +18,8 @@
 #include 
 
 #include "ignite/odbc/log.h"
-#include "ignite/odbc/odbc_error.h"
-#include "ignite/odbc/query/table_metadata_query.h"
+#include "ignite/odbc/query/primary_keys_query.h"
 #include "ignite/odbc/sql_connection.h"
-#include "ignite/odbc/string_utils.h"
 #include "ignite/odbc/type_traits.h"
 
 namespace {
@@ -36,26 +34,65 @@ enum class result_column {
 /** Table name. */
 TABLE_NAME,
 
-/** Table type. */
-TABLE_TYPE,
+/** Column name. */
+COLUMN_NAME,
 
-/** A description of the column. */
-REMARKS
+/** Column sequence number in key. */
+KEY_SEQ,
+
+/** Primary key name. */
+PK_NAME
 };
 
-} // namespace
+using namespace ignite;
+
+/**
+ * Reads primary keys.
+ *
+ * @param reader Reader.
+ * @return Primary keys meta.
+ */
+primary_key_meta_vector read_meta(protocol::reader &reader) {
+auto has_no_data = reader.try_read_nil();
+if (has_no_data)
+return {};
+
+auto tables_num = reader.read_int32();
+
+primary_key_meta_vector keys;
+keys.reserve(tables_num);
+
+for (std::int32_t table_idx = 0; table_idx < tables_num; ++table_idx) {
+auto schema_name = reader.read_string();
+auto table_name = reader.read_string();
+auto key_name = reader.read_string();
+
+auto have_no_fields = reader.try_read_nil();
+if (have_no_fields)
+continue;
+
+auto fields_num = reader.read_int32();
+for (std::int32_t field_idx = 0; field_idx < fields_num; ++field_idx) {
+auto field_name = reader.read_string();
+
+keys.emplace_back("", schema_name, table_name, field_name, 
std::int16_t(field_idx + 1), key_name);
+}
+}
+
+return keys;
+}
+
+} // anonymous namespace
 
 namespace ignite {
 
-table_metadata_query::table_metadata_query(diagnosable_adapter &diag, 
sql_connection &connection,

[ignite-3] branch main updated: IGNITE-20636 Add to the MakeIndexAvailableCommand the ability to use only the indexId (#2691)

2023-10-16 Thread tkalkirill
This is an automated email from the ASF dual-hosted git repository.

tkalkirill pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/ignite-3.git


The following commit(s) were added to refs/heads/main by this push:
 new 05c2c0fc85 IGNITE-20636 Add to the MakeIndexAvailableCommand the 
ability to use only the indexId (#2691)
05c2c0fc85 is described below

commit 05c2c0fc8573b3bce8893588b573db6a3844663d
Author: Kirill Tkalenko 
AuthorDate: Tue Oct 17 09:20:08 2023 +0300

IGNITE-20636 Add to the MakeIndexAvailableCommand the ability to use only 
the indexId (#2691)
---
 .../apache/ignite/internal/catalog/Catalog.java| 13 +++-
 .../IndexAlreadyAvailableValidationException.java  |  3 +-
 .../catalog/IndexNotFoundValidationException.java  |  2 +
 .../internal/catalog/commands/CatalogUtils.java| 22 +-
 .../catalog/commands/CreateTableCommand.java   |  1 +
 .../commands/MakeIndexAvailableCommand.java| 86 +-
 .../commands/MakeIndexAvailableCommandBuilder.java |  8 +-
 .../descriptors/CatalogHashIndexDescriptor.java|  9 ++-
 .../descriptors/CatalogIndexDescriptor.java| 20 +++--
 .../descriptors/CatalogSortedIndexDescriptor.java  |  9 ++-
 .../descriptors/CatalogTableDescriptor.java|  8 ++
 .../internal/catalog/events/CatalogEvent.java  |  2 +-
 .../internal/catalog/storage/AlterColumnEntry.java |  1 +
 .../internal/catalog/storage/DropColumnsEntry.java |  1 +
 .../catalog/storage/MakeIndexAvailableEntry.java   | 79 +++-
 .../internal/catalog/storage/NewColumnsEntry.java  |  1 +
 .../internal/catalog/CatalogManagerSelfTest.java   | 22 --
 .../MakeIndexAvailableCommandValidationTest.java   | 80 ++--
 .../RebalanceUtilUpdateAssignmentsTest.java|  1 +
 .../index/IndexAvailabilityController.java | 17 ++---
 .../index/IndexAvailabilityControllerTest.java |  4 +-
 .../internal/schema/CatalogSchemaManagerTest.java  |  8 +-
 .../CatalogToSchemaDescriptorConverterTest.java|  1 +
 .../engine/schema/CatalogSqlSchemaManagerTest.java |  5 +-
 .../storage/AbstractMvTableStorageTest.java|  6 +-
 .../index/AbstractHashIndexStorageTest.java|  2 +-
 .../storage/index/AbstractIndexStorageTest.java|  2 +
 .../index/AbstractSortedIndexStorageTest.java  |  2 +-
 .../replication/PartitionReplicaListenerTest.java  |  2 +-
 .../distributed/schema/FullTableSchemaTest.java|  2 +-
 30 files changed, 236 insertions(+), 183 deletions(-)

diff --git 
a/modules/catalog/src/main/java/org/apache/ignite/internal/catalog/Catalog.java 
b/modules/catalog/src/main/java/org/apache/ignite/internal/catalog/Catalog.java
index c52416f0f9..f214260376 100644
--- 
a/modules/catalog/src/main/java/org/apache/ignite/internal/catalog/Catalog.java
+++ 
b/modules/catalog/src/main/java/org/apache/ignite/internal/catalog/Catalog.java
@@ -57,6 +57,8 @@ public class Catalog {
 private final Map schemasByName;
 private final Map zonesByName;
 
+@IgniteToStringExclude
+private final Int2ObjectMap schemasById;
 @IgniteToStringExclude
 private final Int2ObjectMap tablesById;
 @IgniteToStringExclude
@@ -88,9 +90,10 @@ public class Catalog {
 Objects.requireNonNull(schemas, "schemas");
 Objects.requireNonNull(zones, "zones");
 
-this.schemasByName = schemas.stream().collect(toMapByName());
-this.zonesByName = zones.stream().collect(toMapByName());
+schemasByName = schemas.stream().collect(toMapByName());
+zonesByName = zones.stream().collect(toMapByName());
 
+schemasById = schemas.stream().collect(toMapById());
 tablesById = schemas.stream().flatMap(s -> 
Arrays.stream(s.tables())).collect(toMapById());
 indexesById = schemas.stream().flatMap(s -> 
Arrays.stream(s.indexes())).collect(toMapById());
 zonesById = zones.stream().collect(toMapById());
@@ -108,10 +111,14 @@ public class Catalog {
 return objectIdGen;
 }
 
-public CatalogSchemaDescriptor schema(String name) {
+public @Nullable CatalogSchemaDescriptor schema(String name) {
 return schemasByName.get(name);
 }
 
+public @Nullable CatalogSchemaDescriptor schema(int schemaId) {
+return schemasById.get(schemaId);
+}
+
 public Collection schemas() {
 return schemasByName.values();
 }
diff --git 
a/modules/catalog/src/main/java/org/apache/ignite/internal/catalog/IndexAlreadyAvailableValidationException.java
 
b/modules/catalog/src/main/java/org/apache/ignite/internal/catalog/IndexAlreadyAvailableValidationException.java
index 7d1f28ff3c..7c5302cbcc 100644
--- 
a/modules/catalog/src/main/java/org/apache/ignite/internal/catalog/IndexAlreadyAvailableValidationException.java
+++ 
b/modules/catalog/src/main/java/org/apache/ignite/internal/catalog/IndexAlreadyAvailableValidationException.java
@@ -20,8 +20,7 @@ package org.apache.ignite.internal.catalog;
 import org.apache.i

[ignite-3] branch main updated: IGNITE-20116 Linearize storage updates with safeTime adjustment rules (#2689)

2023-10-16 Thread sanpwc
This is an automated email from the ASF dual-hosted git repository.

sanpwc 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 884ee2be68 IGNITE-20116 Linearize storage updates with safeTime 
adjustment rules (#2689)
884ee2be68 is described below

commit 884ee2be68175909a47791741ac0bf79256d62ee
Author: Alexander Lapin 
AuthorDate: Tue Oct 17 08:57:57 2023 +0300

IGNITE-20116 Linearize storage updates with safeTime adjustment rules 
(#2689)
---
 .../rebalance/ItRebalanceRecoveryTest.java |   2 -
 ...ItTxDistributedTestThreeNodesThreeReplicas.java |   8 -
 ...butedTestThreeNodesThreeReplicasCollocated.java |   8 -
 .../replicator/PartitionReplicaListener.java   | 781 -
 .../replication/PartitionReplicaListenerTest.java  |   1 +
 5 files changed, 434 insertions(+), 366 deletions(-)

diff --git 
a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/rebalance/ItRebalanceRecoveryTest.java
 
b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/rebalance/ItRebalanceRecoveryTest.java
index 7864bec010..f1af298c4b 100644
--- 
a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/rebalance/ItRebalanceRecoveryTest.java
+++ 
b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/rebalance/ItRebalanceRecoveryTest.java
@@ -26,13 +26,11 @@ import 
org.apache.ignite.internal.ClusterPerTestIntegrationTest;
 import org.apache.ignite.internal.storage.MvPartitionStorage;
 import org.apache.ignite.internal.table.distributed.TableManager;
 import org.apache.ignite.internal.test.WatchListenerInhibitor;
-import org.junit.jupiter.api.Disabled;
 import org.junit.jupiter.api.Test;
 
 /**
  * Tests for recovery of the rebalance procedure.
  */
-@Disabled("https://issues.apache.org/jira/browse/IGNITE-20116";)
 public class ItRebalanceRecoveryTest extends ClusterPerTestIntegrationTest {
 @Override
 protected int initialNodes() {
diff --git 
a/modules/table/src/integrationTest/java/org/apache/ignite/distributed/ItTxDistributedTestThreeNodesThreeReplicas.java
 
b/modules/table/src/integrationTest/java/org/apache/ignite/distributed/ItTxDistributedTestThreeNodesThreeReplicas.java
index c4402aea9d..930e138567 100644
--- 
a/modules/table/src/integrationTest/java/org/apache/ignite/distributed/ItTxDistributedTestThreeNodesThreeReplicas.java
+++ 
b/modules/table/src/integrationTest/java/org/apache/ignite/distributed/ItTxDistributedTestThreeNodesThreeReplicas.java
@@ -30,7 +30,6 @@ import 
org.apache.ignite.internal.tx.impl.ReadWriteTransactionImpl;
 import org.apache.ignite.raft.jraft.rpc.RpcRequests;
 import org.apache.ignite.raft.jraft.rpc.RpcRequests.AppendEntriesRequest;
 import org.junit.jupiter.api.AfterEach;
-import org.junit.jupiter.api.Disabled;
 import org.junit.jupiter.api.Test;
 import org.junit.jupiter.api.TestInfo;
 
@@ -59,13 +58,6 @@ public class ItTxDistributedTestThreeNodesThreeReplicas 
extends ItTxDistributedT
 return 3;
 }
 
-/** {@inheritDoc} */
-@Disabled("https://issues.apache.org/jira/browse/IGNITE-20116";)
-@Override
-public void testBalance() throws InterruptedException {
-super.testBalance();
-}
-
 @Override
 @AfterEach
 public void after() throws Exception {
diff --git 
a/modules/table/src/integrationTest/java/org/apache/ignite/distributed/ItTxDistributedTestThreeNodesThreeReplicasCollocated.java
 
b/modules/table/src/integrationTest/java/org/apache/ignite/distributed/ItTxDistributedTestThreeNodesThreeReplicasCollocated.java
index e128acac09..79a6080e6b 100644
--- 
a/modules/table/src/integrationTest/java/org/apache/ignite/distributed/ItTxDistributedTestThreeNodesThreeReplicasCollocated.java
+++ 
b/modules/table/src/integrationTest/java/org/apache/ignite/distributed/ItTxDistributedTestThreeNodesThreeReplicasCollocated.java
@@ -25,7 +25,6 @@ import java.util.UUID;
 import org.apache.ignite.internal.tx.TxState;
 import org.apache.ignite.internal.tx.impl.ReadWriteTransactionImpl;
 import org.junit.jupiter.api.BeforeEach;
-import org.junit.jupiter.api.Disabled;
 import org.junit.jupiter.api.Test;
 import org.junit.jupiter.api.TestInfo;
 
@@ -47,13 +46,6 @@ public class 
ItTxDistributedTestThreeNodesThreeReplicasCollocated extends ItTxDi
 return false;
 }
 
-/** {@inheritDoc} */
-@Disabled("https://issues.apache.org/jira/browse/IGNITE-20116";)
-@Override
-public void testBalance() throws InterruptedException {
-super.testBalance();
-}
-
 /** {@inheritDoc} */
 @BeforeEach
 @Override public void before() throws Exception {
diff --git 
a/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/replicator/PartitionReplicaListener.java
 
b/modules/table/src/main/java/org/apache/ignite/internal/table/distributed/replicator/PartitionReplicaListener.java
index 8311d76853..549499fd3e 100644
-

[ignite-3] branch main updated: IGNITE-20444 Sql. Add restrictions for execution tx related statements with single statement mode (#2683)

2023-10-16 Thread zstan
This is an automated email from the ASF dual-hosted git repository.

zstan 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 e443b6f862 IGNITE-20444 Sql. Add restrictions for execution tx related 
statements with single statement mode (#2683)
e443b6f862 is described below

commit e443b6f8627f3b17c8b20b3f4b42b96036a827be
Author: Max Zhuravkov 
AuthorDate: Tue Oct 17 08:51:23 2023 +0300

IGNITE-20444 Sql. Add restrictions for execution tx related statements with 
single statement mode (#2683)
---
 .../client/handler/JdbcQueryEventHandlerImpl.java  |   2 +-
 .../benchmark/AbstractOneNodeBenchmark.java|   2 +-
 .../internal/sql/engine/ItMultistatementTest.java  | 120 +
 .../ignite/internal/sql/api/SessionImpl.java   |   2 +-
 .../internal/sql/engine/SqlQueryProcessor.java |   6 ++
 .../ignite/internal/sql/engine/SqlQueryType.java   |  11 +-
 .../ignite/internal/sql/engine/util/Commons.java   |   8 ++
 .../internal/sql/engine/util/QueryCheckerImpl.java |   2 +-
 8 files changed, 148 insertions(+), 5 deletions(-)

diff --git 
a/modules/client-handler/src/main/java/org/apache/ignite/client/handler/JdbcQueryEventHandlerImpl.java
 
b/modules/client-handler/src/main/java/org/apache/ignite/client/handler/JdbcQueryEventHandlerImpl.java
index fabcb33355..24dbdd0e40 100644
--- 
a/modules/client-handler/src/main/java/org/apache/ignite/client/handler/JdbcQueryEventHandlerImpl.java
+++ 
b/modules/client-handler/src/main/java/org/apache/ignite/client/handler/JdbcQueryEventHandlerImpl.java
@@ -180,7 +180,7 @@ public class JdbcQueryEventHandlerImpl implements 
JdbcQueryEventHandler {
 private QueryContext createQueryContext(JdbcStatementType stmtType, 
@Nullable Transaction tx) {
 switch (stmtType) {
 case ANY_STATEMENT_TYPE:
-return QueryContext.create(SqlQueryType.ALL, tx);
+return QueryContext.create(SqlQueryType.SINGLE_STMT_TYPES, tx);
 case SELECT_STATEMENT_TYPE:
 return QueryContext.create(SELECT_STATEMENT_QUERIES, tx);
 case UPDATE_STATEMENT_TYPE:
diff --git 
a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/benchmark/AbstractOneNodeBenchmark.java
 
b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/benchmark/AbstractOneNodeBenchmark.java
index 3de017059b..e2dbacc0aa 100644
--- 
a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/benchmark/AbstractOneNodeBenchmark.java
+++ 
b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/benchmark/AbstractOneNodeBenchmark.java
@@ -111,7 +111,7 @@ public class AbstractOneNodeBenchmark {
 + ");";
 
 try {
-var context = QueryContext.create(SqlQueryType.ALL);
+var context = QueryContext.create(SqlQueryType.SINGLE_STMT_TYPES);
 
 getAllFromCursor(
 await(queryEngine.querySingleAsync(sessionId, context, 
clusterNode.transactions(), sql))
diff --git 
a/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItMultistatementTest.java
 
b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItMultistatementTest.java
new file mode 100644
index 00..fadefebd9e
--- /dev/null
+++ 
b/modules/runner/src/integrationTest/java/org/apache/ignite/internal/sql/engine/ItMultistatementTest.java
@@ -0,0 +1,120 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *  http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.sql.engine;
+
+import static org.apache.ignite.lang.ErrorGroups.Sql.STMT_VALIDATION_ERR;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.Matchers.containsString;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertInstanceOf;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+
+import java.util.Arrays;
+import java.util.concurrent.CompletionException;
+import java.util.stream.Stream;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.sql.IgniteSql;
+import org.apache.ignite.sql.Session

[ignite-3] branch main updated: IGNITE-20657 At the checkpoint ArrayIndexOutOfBoundsException (#2696)

2023-10-16 Thread tkalkirill
This is an automated email from the ASF dual-hosted git repository.

tkalkirill pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/ignite-3.git


The following commit(s) were added to refs/heads/main by this push:
 new c962aa1bbb IGNITE-20657 At the checkpoint 
ArrayIndexOutOfBoundsException (#2696)
c962aa1bbb is described below

commit c962aa1bbb94e73a4d8ce2403aad3d629dd55666
Author: Kirill Tkalenko 
AuthorDate: Mon Oct 16 21:47:29 2023 +0300

IGNITE-20657 At the checkpoint ArrayIndexOutOfBoundsException (#2696)
---
 .../persistence/store/FilePageStore.java   | 121 -
 .../persistence/store/FilePageStoreTest.java   |  16 +++
 2 files changed, 86 insertions(+), 51 deletions(-)

diff --git 
a/modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/store/FilePageStore.java
 
b/modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/store/FilePageStore.java
index 7d1c9cb991..041c7c861b 100644
--- 
a/modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/store/FilePageStore.java
+++ 
b/modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/persistence/store/FilePageStore.java
@@ -17,6 +17,7 @@
 
 package org.apache.ignite.internal.pagememory.persistence.store;
 
+import static java.util.Collections.unmodifiableList;
 import static org.apache.ignite.internal.pagememory.util.PageIdUtils.pageIndex;
 
 import java.io.IOException;
@@ -24,12 +25,11 @@ import java.lang.invoke.MethodHandles;
 import java.lang.invoke.VarHandle;
 import java.nio.ByteBuffer;
 import java.nio.file.Path;
+import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Comparator;
-import java.util.Iterator;
 import java.util.List;
 import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.CopyOnWriteArrayList;
 import java.util.function.IntFunction;
 import java.util.function.Supplier;
 import org.apache.ignite.internal.lang.IgniteInternalCheckedException;
@@ -53,6 +53,8 @@ public class FilePageStore implements PageStore {
 
 private static final VarHandle NEW_DELTA_FILE_PAGE_STORE_IO_FUTURE;
 
+private static final VarHandle DELTA_FILE_PAGE_STORE_IOS;
+
 static {
 try {
 PAGE_COUNT = 
MethodHandles.lookup().findVarHandle(FilePageStore.class, "pageCount", 
int.class);
@@ -62,6 +64,8 @@ public class FilePageStore implements PageStore {
 "newDeltaFilePageStoreIoFuture",
 CompletableFuture.class
 );
+
+DELTA_FILE_PAGE_STORE_IOS = 
MethodHandles.lookup().findVarHandle(FilePageStore.class, 
"deltaFilePageStoreIos", List.class);
 } catch (ReflectiveOperationException e) {
 throw new ExceptionInInitializerError(e);
 }
@@ -88,8 +92,8 @@ public class FilePageStore implements PageStore {
 /** New page allocation listener. */
 private volatile @Nullable PageAllocationListener pageAllocationListener;
 
-/** Delta file page store IOs. */
-private final List deltaFilePageStoreIos;
+/** Delta file page store IOs. Copy-on-write list. */
+private volatile List deltaFilePageStoreIos;
 
 /** Future with a new delta file page store. */
 private volatile @Nullable CompletableFuture 
newDeltaFilePageStoreIoFuture;
@@ -112,10 +116,9 @@ public class FilePageStore implements PageStore {
 }
 
 this.filePageStoreIo = filePageStoreIo;
-this.deltaFilePageStoreIos = new 
CopyOnWriteArrayList<>(Arrays.asList(deltaFilePageStoreIos));
+this.deltaFilePageStoreIos = Arrays.asList(deltaFilePageStoreIos);
 }
 
-/** {@inheritDoc} */
 @Override
 public void stop(boolean clean) throws IgniteInternalCheckedException {
 filePageStoreIo.stop(clean);
@@ -125,7 +128,6 @@ public class FilePageStore implements PageStore {
 }
 }
 
-/** {@inheritDoc} */
 @Override
 public int allocatePage() throws IgniteInternalCheckedException {
 ensure();
@@ -141,7 +143,6 @@ public class FilePageStore implements PageStore {
 return pageIdx;
 }
 
-/** {@inheritDoc} */
 @Override
 public int pages() {
 return pageCount;
@@ -181,7 +182,6 @@ public class FilePageStore implements PageStore {
 filePageStoreIo.read(pageId, filePageStoreIo.pageOffset(pageId), 
pageBuf, keepCrc);
 }
 
-/** {@inheritDoc} */
 @Override
 public void read(long pageId, ByteBuffer pageBuf, boolean keepCrc) throws 
IgniteInternalCheckedException {
 assert pageIndex(pageId) <= pageCount : "pageIdx=" + pageIndex(pageId) 
+ ", pageCount=" + pageCount;
@@ -189,7 +189,6 @@ public class FilePageStore implements PageStore {
 readWithoutPageIdCheck(pageId, pageBuf, keepCrc);
 }
 
-/** {@inheritDoc} */
 @Override
 public void write(long pageId, ByteBuffer pageBuf, boolean calculateCrc) 
throws IgniteInternalCheckedException 

[ignite] branch master updated: IGNITE-20579 Finally restrict atomic operations inside the transaction (remove the system property) (#10979)

2023-10-16 Thread av
This is an automated email from the ASF dual-hosted git repository.

av pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/ignite.git


The following commit(s) were added to refs/heads/master by this push:
 new 63476a7a4bf IGNITE-20579 Finally restrict atomic operations inside the 
transaction (remove the system property) (#10979)
63476a7a4bf is described below

commit 63476a7a4bf917688af4a1c8d9a24829fa0736dc
Author: Anton Vinogradov 
AuthorDate: Mon Oct 16 21:10:22 2023 +0300

IGNITE-20579 Finally restrict atomic operations inside the transaction 
(remove the system property) (#10979)
---
 .../main/java/org/apache/ignite/IgniteCache.java   |  11 --
 .../org/apache/ignite/IgniteSystemProperties.java  |   8 -
 .../processors/cache/CacheOperationContext.java|  62 +---
 .../cache/GatewayProtectedCacheProxy.java  |  22 ---
 .../processors/cache/GridCacheAdapter.java |  28 +---
 .../processors/cache/GridCacheGateway.java |  31 ++--
 .../processors/cache/GridCacheProxyImpl.java   |  26 +---
 .../processors/cache/IgniteCacheProxy.java |   3 -
 .../processors/cache/IgniteCacheProxyImpl.java |   5 -
 .../processors/cache/IgniteInternalCache.java  |   5 -
 .../datastructures/GridCacheQueueAdapter.java  |   4 +-
 ...acheJdbcStoreAbstractMultithreadedSelfTest.java |   8 +-
 .../internal/IgniteClientReconnectCacheTest.java   |   8 +-
 .../cache/CacheGetsDistributionAbstractTest.java   |  32 ++--
 .../cache/CacheReadThroughRestartSelfTest.java |   8 +-
 .../GridAbstractCacheInterceptorRebalanceTest.java |   4 +-
 .../cache/GridCacheAbstractFullApiSelfTest.java|  69 ++---
 .../cache/GridCacheAbstractLocalStoreSelfTest.java |  12 +-
 .../cache/GridCacheAbstractSelfTest.java   |   2 +-
 .../IgniteCacheConfigVariationsFullApiTest.java|   3 +-
 ...gniteStartCacheInTransactionAtomicSelfTest.java |  32 
 .../IgniteStartCacheInTransactionSelfTest.java |  12 +-
 .../CacheGetInsideLockChangingTopologyTest.java|  66 +
 .../GridCacheAbstractNodeRestartSelfTest.java  |   4 +-
 .../distributed/IgniteCacheTxIteratorSelfTest.java |  21 ++-
 .../near/GridNearCacheStoreUpdateTest.java |  18 +--
 .../ClientAffinityAssignmentWithBaselineTest.java  |   5 +-
 .../db/wal/reader/IgniteWalReaderTest.java |  17 ++-
 .../transactions/AtomicOperationsInTxTest.java | 162 +
 .../TxOptimisticDeadlockDetectionTest.java |   2 +-
 .../TxPessimisticDeadlockDetectionTest.java|   3 +-
 .../junits/common/GridCommonAbstractTest.java  |   4 +-
 .../junits/multijvm/IgniteCacheProcessProxy.java   |   5 -
 .../ignite/testsuites/IgniteCacheTestSuite4.java   |   2 -
 .../internal/processors/query/h2/dml/DmlUtils.java |   2 +-
 .../processors/cache/IgniteCacheGroupsSqlTest.java |   6 +-
 .../cache/ttl/CacheTtlAbstractSelfTest.java|  21 +--
 .../ExpiryCacheHolderTest.cs   |   7 -
 .../Cache/CacheTestAsyncWrapper.cs |  12 --
 .../dotnet/Apache.Ignite.Core/Cache/ICache.cs  |  14 --
 .../Apache.Ignite.Core/Impl/Cache/CacheImpl.cs |  19 ---
 41 files changed, 228 insertions(+), 557 deletions(-)

diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteCache.java 
b/modules/core/src/main/java/org/apache/ignite/IgniteCache.java
index e06bcd693a1..701caa3f237 100644
--- a/modules/core/src/main/java/org/apache/ignite/IgniteCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/IgniteCache.java
@@ -242,17 +242,6 @@ public interface IgniteCache extends 
javax.cache.Cache, IgniteAsyncS
  */
 public  IgniteCache withKeepBinary();
 
-/**
- * If you want to use atomic operations inside transactions you should 
allow it before transaction start.
- * To enable this behavior by default you can set system property
- * {@link IgniteSystemProperties#IGNITE_ALLOW_ATOMIC_OPS_IN_TX 
IGNITE_ALLOW_ATOMIC_OPS_IN_TX} to {@code true}.
- *
- * @param  Type of the cache value.
- * @param  Type of the cache key.
- * @return Cache with atomic operations allowed in transactions.
- */
-public  IgniteCache withAllowAtomicOpsInTx();
-
 /**
  * Executes {@link #localLoadCache(IgniteBiPredicate, Object...)} on all 
cache nodes.
  *
diff --git 
a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java 
b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
index 6698fcb94b1..4a17a95c2c4 100644
--- a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
+++ b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
@@ -596,14 +596,6 @@ public final class IgniteSystemProperties {
 @SystemProperty("Disables performance suggestions output on start")
 public static final String IGNITE_PERFORMANCE_SUGGESTIONS_DISABLED = 
"IGNITE_PERFORMANCE_SUGGESTIONS_DISABLED";
 
-/**
- * Flag indicating whether at

[ignite-3] branch main updated: IGNITE-20512 Remove port range from HTTP server (#2673)

2023-10-16 Thread apkhmv
This is an automated email from the ASF dual-hosted git repository.

apkhmv 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 604840c711 IGNITE-20512 Remove port range from HTTP server (#2673)
604840c711 is described below

commit 604840c71157fbe287f9093ab636a4802c7ca557
Author: Aleksandr Pakhomov 
AuthorDate: Mon Oct 16 19:36:44 2023 +0400

IGNITE-20512 Remove port range from HTTP server (#2673)

* Remove port range from HTTP server configuration
* Update junit5 version to avoid jar hell.
---
 gradle/libs.versions.toml  |  2 +-
 .../org/apache/ignite/internal/NodeConfig.java |  8 ++-
 .../internal/cli/CliIntegrationTestBase.java   |  4 ++
 .../internal/rest/ItGeneratedRestClientTest.java   |  3 +-
 .../resources/hardcoded-ports-config.json  |  3 +-
 .../testframework/IntegrationTestBase.java | 30 +++
 .../apache/ignite/internal/rest/RestComponent.java | 60 +-
 .../configuration/RestConfigurationSchema.java |  5 --
 .../configuration/RestSslConfigurationSchema.java  |  5 --
 .../ignite/internal/BaseIgniteRestartTest.java | 14 -
 .../internal/ClusterPerTestIntegrationTest.java| 12 ++---
 .../cluster/management/ItClusterInitTest.java  |  3 +-
 .../internal/compute/ItLogicalTopologyTest.java|  3 +-
 .../zones/ItDistributionZonesFilterTest.java   |  3 +-
 .../raftsnapshot/ItTableRaftSnapshotsTest.java |  3 +-
 .../ignite/internal/rest/AbstractRestTestBase.java | 40 ++-
 .../{ItPortRangeTest.java => ItRestPortsTest.java} | 10 ++--
 .../runner/app/AbstractSchemaChangeTest.java   |  9 ++--
 .../internal/runner/app/ItDataSchemaSyncTest.java  |  9 ++--
 .../runner/app/ItIgniteNodeRestartTest.java|  4 +-
 .../ignite/internal/runner/app/ItIgnitionTest.java |  9 ++--
 .../internal/runner/app/ItTablesApiTest.java   |  9 ++--
 .../runner/app/PlatformTestNodeRunner.java | 12 +++--
 .../app/client/ItAbstractThinClientTest.java   |  6 ++-
 .../sql/engine/ClusterPerClassIntegrationTest.java | 11 +++-
 .../ignite/internal/sqllogic/ItSqlLogicTest.java   |  9 +++-
 .../org/apache/ignite/internal/ssl/ItSslTest.java  | 54 ---
 .../ignite-config-rest-port-not-default.json   |  3 +-
 .../java/org/apache/ignite/internal/Cluster.java   | 13 -
 29 files changed, 222 insertions(+), 134 deletions(-)

diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml
index 320789e57e..c3408b096f 100644
--- a/gradle/libs.versions.toml
+++ b/gradle/libs.versions.toml
@@ -30,7 +30,7 @@ javax = "1.3.2"
 jetbrainsAnnotations = "20.1.0"
 jline = "3.21.0"
 jmh = "1.35"
-junit5 = "5.9.1"
+junit5 = "5.9.3"
 junitPioneer = "2.0.1"
 jsr305 = "3.0.2"
 okhttp = "4.9.1"
diff --git 
a/modules/cli/src/integrationTest/java/org/apache/ignite/internal/NodeConfig.java
 
b/modules/cli/src/integrationTest/java/org/apache/ignite/internal/NodeConfig.java
index cd533dd407..798233bc45 100644
--- 
a/modules/cli/src/integrationTest/java/org/apache/ignite/internal/NodeConfig.java
+++ 
b/modules/cli/src/integrationTest/java/org/apache/ignite/internal/NodeConfig.java
@@ -40,8 +40,10 @@ public class NodeConfig {
 + "},\n"
 + "  },\n"
 + "  clientConnector.port: {} ,\n"
-+ "  rest: {"
++ "  rest: {\n"
++ "port: {}\n"
 + "ssl: {\n"
++ "  port: {},\n"
 + "  enabled: true,\n"
 + "  keyStore: {\n"
 + "path: \"" +  escapeWindowsPath(resolvedKeystorePath) + 
"\",\n"
@@ -77,6 +79,10 @@ public class NodeConfig {
 + "password: " + trustStorePassword + "\n"
 + "  }\n"
 + "}\n"
++ "  },\n"
++ "  rest: {\n"
++ "port: {},\n"
++ "ssl.port: {}\n"
 + "  }\n"
 + "}";
 }
diff --git 
a/modules/cli/src/integrationTest/java/org/apache/ignite/internal/cli/CliIntegrationTestBase.java
 
b/modules/cli/src/integrationTest/java/org/apache/ignite/internal/cli/CliIntegrationTestBase.java
index 8d822429cf..b9bd5b99dd 100644
--- 
a/modules/cli/src/integrationTest/java/org/apache/ignite/internal/cli/CliIntegrationTestBase.java
+++ 
b/modules/cli/src/integrationTest/java/org/apache/ignite/internal/cli/CliIntegrationTestBase.java
@@ -55,6 +55,10 @@ public abstract class CliIntegrationTestBase extends 
IntegrationTestBase {
 + "}\n"
 + "  },\n"
 + "  clientConnector: { port:{} }\n"
++ "  rest: {\n"
++ "port: {}\n"
++ "ssl.port: {}\n"
++ "  }\n"
 + "}";
 
 
diff --git 
a/modules/cli/src/integrationTest/java/org/apache/ignite/internal/rest/ItGeneratedRestClientTest.java
 
b/modules/cli/src/integrationTes

[ignite-3] branch main updated (c44bbbb229 -> 956c01325d)

2023-10-16 Thread isapego
This is an automated email from the ASF dual-hosted git repository.

isapego pushed a change to branch main
in repository https://gitbox.apache.org/repos/asf/ignite-3.git


from c44229 IGNITE-20387: Remap most exceptions to SqlExceptions for 
SQL API (#2613)
 add 956c01325d IGNITE-19217 Implement foreign keys query for ODBC  (#2692)

No new revisions were added by this update.

Summary of changes:
 .../platforms/cpp/ignite/client/detail/utils.cpp   |  4 +-
 modules/platforms/cpp/ignite/odbc/CMakeLists.txt   |  1 +
 .../cpp/ignite/odbc/query/column_metadata_query.h  |  2 +-
 .../cpp/ignite/odbc/query/foreign_keys_query.cpp   | 88 ++
 .../{type_info_query.h => foreign_keys_query.h}| 50 +++-
 modules/platforms/cpp/ignite/odbc/query/query.h|  3 +
 .../cpp/ignite/odbc/query/table_metadata_query.cpp | 20 ++---
 .../cpp/ignite/odbc/query/table_metadata_query.h   |  6 +-
 .../cpp/ignite/odbc/query/type_info_query.h|  2 +-
 .../platforms/cpp/ignite/odbc/sql_statement.cpp| 14 ++--
 modules/platforms/cpp/ignite/odbc/sql_statement.h  |  2 +-
 modules/platforms/cpp/ignite/odbc/type_traits.h|  4 +-
 .../client-test/key_value_binary_view_test.cpp |  6 +-
 .../tests/client-test/record_binary_view_test.cpp  |  6 +-
 .../cpp/tests/client-test/record_view_test.cpp |  5 +-
 .../cpp/tests/odbc-test/api_robustness_test.cpp|  4 +-
 16 files changed, 160 insertions(+), 57 deletions(-)
 create mode 100644 
modules/platforms/cpp/ignite/odbc/query/foreign_keys_query.cpp
 copy modules/platforms/cpp/ignite/odbc/query/{type_info_query.h => 
foreign_keys_query.h} (65%)



[ignite-3] branch main updated: IGNITE-20387: Remap most exceptions to SqlExceptions for SQL API (#2613)

2023-10-16 Thread korlov
This is an automated email from the ASF dual-hosted git repository.

korlov 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 c44229 IGNITE-20387: Remap most exceptions to SqlExceptions for 
SQL API (#2613)
c44229 is described below

commit c44229d44737fdb99fa29c3a8b4d5e4cc248
Author: ygerzhedovich <41903880+ygerzhedov...@users.noreply.github.com>
AuthorDate: Mon Oct 16 16:50:50 2023 +0300

IGNITE-20387: Remap most exceptions to SqlExceptions for SQL API (#2613)
---
 .../internal/lang/IgniteExceptionMapperUtil.java   |  33 +-
 .../ignite/internal/sql/SyncResultSetAdapter.java  |   4 +-
 .../dotnet/Apache.Ignite.Tests/Sql/SqlTests.cs |  14 +-
 .../dotnet/Apache.Ignite/Internal/Sql/Sql.cs   |   2 +-
 .../internal/runner/app/ItTablesApiTest.java   |  31 +-
 ...nchronousApiTest.java => ItSqlApiBaseTest.java} | 587 +++---
 .../internal/sql/api/ItSqlAsynchronousApiTest.java | 851 ++---
 .../sql/api/ItSqlClientAsynchronousApiTest.java|   6 +
 .../sql/api/ItSqlClientSynchronousApiTest.java |  24 +-
 .../internal/sql/api/ItSqlSynchronousApiTest.java  | 574 ++
 .../internal/sql/engine/ItCreateTableDdlTest.java  |   6 +-
 .../internal/lang/SqlExceptionMapperUtil.java  |  64 ++
 .../ignite/internal/sql/api/SessionImpl.java   |  12 +-
 .../internal/sql/engine/AsyncSqlCursorImpl.java|   4 +-
 .../sql/engine/prepare/PrepareServiceImpl.java |   4 +-
 .../internal/lang/SqlExceptionMapperUtilTest.java  |  78 ++
 .../internal/sql/engine/util/SqlTestUtils.java |   8 +
 17 files changed, 623 insertions(+), 1679 deletions(-)

diff --git 
a/modules/core/src/main/java/org/apache/ignite/internal/lang/IgniteExceptionMapperUtil.java
 
b/modules/core/src/main/java/org/apache/ignite/internal/lang/IgniteExceptionMapperUtil.java
index e5b43059aa..fbae12e574 100644
--- 
a/modules/core/src/main/java/org/apache/ignite/internal/lang/IgniteExceptionMapperUtil.java
+++ 
b/modules/core/src/main/java/org/apache/ignite/internal/lang/IgniteExceptionMapperUtil.java
@@ -52,8 +52,8 @@ public class IgniteExceptionMapperUtil {
  *
  * @param mapper Exception mapper from internal exception to a public one.
  * @param registeredMappings Already registered mappings.
- * @throws IgniteException If a mapper for the given {@code clazz} already 
registered,
- *  or {@code clazz} represents Java standard exception like {@link 
NullPointerException}, {@link IllegalArgumentException}.
+ * @throws IgniteException If a mapper for the given {@code clazz} already 
registered, or {@code clazz} represents Java standard
+ * exception like {@link NullPointerException}, {@link 
IllegalArgumentException}.
  */
 static void registerMapping(
 IgniteExceptionMapper mapper,
@@ -87,23 +87,30 @@ public class IgniteExceptionMapperUtil {
 if (origin instanceof AssertionError) {
 return new IgniteException(INTERNAL_ERR, origin);
 }
-
 return origin;
 }
 
-IgniteExceptionMapper m = 
EXCEPTION_CONVERTERS.get(origin.getClass());
+Throwable res;
+
+// Try to find appropriate mapper, moving from original class to 
supper-classes step by step.
+Class exceptionClass = origin.getClass();
+IgniteExceptionMapper m;
+while ((m = EXCEPTION_CONVERTERS.get(exceptionClass)) == null && 
exceptionClass != Throwable.class) {
+exceptionClass = exceptionClass.getSuperclass();
+}
+
 if (m != null) {
-Exception mapped = map(m, origin);
+res = map(m, origin);
 
-assert mapped instanceof IgniteException || mapped instanceof 
IgniteCheckedException :
-"Unexpected mapping of internal exception to a public one 
[origin=" + origin + ", mapped=" + mapped + ']';
+assert res instanceof IgniteException || res instanceof 
IgniteCheckedException :
+"Unexpected mapping of internal exception to a public one 
[origin=" + origin + ", mapped=" + res + ']';
 
-return mapped;
+} else {
+res = origin;
 }
 
-if (origin instanceof IgniteException || origin instanceof 
IgniteCheckedException) {
-
-return origin;
+if (res instanceof IgniteException || res instanceof 
IgniteCheckedException) {
+return res;
 }
 
 // There are no exception mappings for the given exception. This case 
should be considered as internal error.
@@ -111,8 +118,8 @@ public class IgniteExceptionMapperUtil {
 }
 
 /**
- * Returns a new CompletableFuture that, when the given {@code origin} 
future completes exceptionally,
- * maps the origin's exception to a public Ignite exception if it is 
needed.
+ * Returns a new CompletableFuture that, when the g

[ignite-3] branch main updated: IGNITE-20578 Implement scan over system view (#2678)

2023-10-16 Thread korlov
This is an automated email from the ASF dual-hosted git repository.

korlov 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 779df2e460 IGNITE-20578 Implement scan over system view (#2678)
779df2e460 is described below

commit 779df2e4607e2f2e2b6280ee15ef58d8aee42b61
Author: korlov42 
AuthorDate: Mon Oct 16 14:38:11 2023 +0300

IGNITE-20578 Implement scan over system view (#2678)
---
 modules/catalog/build.gradle   |   2 +
 .../internal/catalog/CatalogManagerImpl.java   | 111 +-
 .../descriptors/CatalogSystemViewDescriptor.java   |   4 +-
 .../internal/catalog/CatalogSystemViewTest.java|   4 +-
 .../CreateSystemViewCommandValidationTest.java |   2 +-
 .../ignite/internal/lang}/InternalTuple.java   |   4 +-
 .../ignite/internal/util/SubscriptionUtils.java|  17 +
 .../subscription/IterableToPublisherAdapter.java   | 193 ++
 modules/distribution-zones/build.gradle|   1 +
 modules/runner/build.gradle|   2 +
 .../internal/sql/engine/ItSystemViewsTest.java | 182 ++
 .../org/apache/ignite/internal/app/IgniteImpl.java |   5 +-
 modules/schema/build.gradle|   1 +
 .../apache/ignite/internal/schema/BinaryTuple.java |   2 +-
 .../ignite/internal/schema/BinaryTuplePrefix.java  |   2 +-
 .../ignite/internal/schema/BinaryTupleSchema.java  |  52 ++-
 .../org/apache/ignite/internal/schema/row/Row.java |   1 +
 modules/sql-engine/build.gradle|   1 +
 .../internal/sql/engine/SqlQueryProcessor.java |   2 +-
 .../internal/sql/engine/exec/RowHandler.java   |   2 +-
 .../sql/engine/exec/ScannableDataSource.java   |   2 +-
 .../internal/sql/engine/exec/SqlRowHandler.java|   2 +-
 .../sql/engine/exec/TableRowConverterImpl.java |   2 +-
 .../sql/engine/exec/rel/DataSourceScanNode.java|   2 +-
 .../sql/engine/schema/CatalogSqlSchemaManager.java |   4 +-
 .../sql/engine/util/AbstractProjectedTuple.java|   2 +-
 .../util/FieldDeserializingProjectedTuple.java |   2 +-
 .../sql/engine/util/FormatAwareProjectedTuple.java |   2 +-
 .../sql/engine/exec/rel/AbstractExecutionTest.java |   2 +-
 .../exec/rel/DataSourceScanNodeSelfTest.java   |   2 +-
 .../sql/engine/framework/ArrayRowHandler.java  |   2 +-
 .../engine/schema/CatalogSqlSchemaManagerTest.java |   4 +-
 .../sql/engine/util/ProjectedTupleTest.java|   2 +-
 modules/storage-api/build.gradle   |   1 +
 .../build.gradle}  |  21 +-
 .../systemview/api}/ClusterSystemView.java |  16 +-
 .../internal/systemview/api}/NodeSystemView.java   |  25 +-
 .../internal/systemview/api}/SystemView.java   |  48 ++-
 .../internal/systemview/api}/SystemViewColumn.java |   2 +-
 .../systemview/api}/SystemViewManager.java |  12 +-
 .../systemview/api/SystemViewProvider.java}|  15 +-
 .../internal/systemview/api}/SystemViews.java  |  10 +-
 .../internal/systemview/api/SystemViewTest.java| 403 +
 modules/system-view/build.gradle   |  17 +-
 .../internal/systemview/SystemViewManagerImpl.java |  91 -
 .../internal/systemview/utils/SystemViewUtils.java |  57 ++-
 .../internal/systemview/SystemViewManagerTest.java | 149 ++--
 .../ignite/internal/systemview/SystemViewTest.java | 278 --
 modules/table/build.gradle |   1 +
 settings.gradle|   2 +
 50 files changed, 1327 insertions(+), 441 deletions(-)

diff --git a/modules/catalog/build.gradle b/modules/catalog/build.gradle
index ab97262744..96c2e643fd 100644
--- a/modules/catalog/build.gradle
+++ b/modules/catalog/build.gradle
@@ -29,6 +29,7 @@ dependencies {
 implementation project(':ignite-configuration')
 implementation project(':ignite-metastorage-api')
 implementation project(':ignite-vault')
+implementation project(':ignite-system-view-api')
 
 implementation libs.jetbrains.annotations
 implementation libs.auto.service.annotations
@@ -49,6 +50,7 @@ dependencies {
 testFixturesImplementation libs.mockito.core
 testFixturesImplementation libs.mockito.junit
 testFixturesImplementation libs.hamcrest.core
+testFixturesImplementation project(':ignite-system-view-api')
 testFixturesImplementation(testFixtures(project(':ignite-core')))
 testFixturesImplementation(testFixtures(project(':ignite-vault')))
 testFixturesImplementation(testFixtures(project(':ignite-metastorage')))
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 faf1316d9b..aab4f0f777 100644
--- 
a/modules/catalog/src/main/java/org/apache/ignite/internal/catalog/CatalogManagerImpl.java
+++ 

[ignite-3] branch main updated (85f4fe5564 -> ce2a2e0559)

2023-10-16 Thread apkhmv
This is an automated email from the ASF dual-hosted git repository.

apkhmv pushed a change to branch main
in repository https://gitbox.apache.org/repos/asf/ignite-3.git


from 85f4fe5564 IGNITE-19276 Implement a mechanism to build indices 
distributively (#2676)
 add ce2a2e0559 IGNITE-20567 Move the 'enabled' flag from the 
authentication configuration to security (#2665)

No new revisions were added by this update.

Summary of changes:
 .../internal/rest/ItGeneratedRestClientTest.java   |  4 +-
 .../cluster-configuration-with-enabled-auth.conf   |  2 +-
 .../ignite/client/handler/ItClientHandlerTest.java | 10 +--
 .../apache/ignite/client/handler/TestServer.java   | 16 ++---
 .../ignite/client/handler/ClientHandlerModule.java | 14 ++--
 .../handler/ClientInboundMessageHandler.java   |  6 +-
 .../ignite/client/ClientAuthenticationTest.java| 14 ++--
 .../ignite/client/TestClientHandlerModule.java | 17 ++---
 .../java/org/apache/ignite/client/TestServer.java  | 24 +++
 .../cluster/ItClusterManagementControllerTest.java |  6 +-
 .../apache/ignite/internal/jdbc/ItJdbcTest.java|  2 +-
 .../rest/ItInitializedClusterRestTest.java |  2 +-
 .../rest/ItNotInitializedClusterRestTest.java  |  4 +-
 .../rest/authentication/ItAuthenticationTest.java  |  6 +-
 .../runner/app/PlatformTestNodeRunner.java | 30 +
 .../org/apache/ignite/internal/app/IgniteImpl.java | 13 ++--
 .../DistributedConfigurationUpdaterTest.java   |  2 +-
 .../authentication/AuthenticationManager.java  |  4 +-
 .../AuthenticationConfigurationSchema.java |  5 --
 .../AuthenticationProvidersValidatorImpl.java  |  4 +-
 .../configuration/SecurityConfigurationSchema.java |  5 ++
 .../authentication/AuthenticationManagerImpl.java  | 11 ++--
 .../AuthenticationManagerImplTest.java | 74 +++---
 .../AuthenticationProvidersValidatorImplTest.java  |  4 +-
 ...onViewEvent.java => StubSecurityViewEvent.java} | 14 ++--
 25 files changed, 148 insertions(+), 145 deletions(-)
 rename 
modules/security/src/test/java/org/apache/ignite/internal/security/authentication/{StubAuthenticationViewEvent.java
 => StubSecurityViewEvent.java} (78%)



[ignite-3] branch main updated: IGNITE-19276 Implement a mechanism to build indices distributively (#2676)

2023-10-16 Thread tkalkirill
This is an automated email from the ASF dual-hosted git repository.

tkalkirill pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/ignite-3.git


The following commit(s) were added to refs/heads/main by this push:
 new 85f4fe5564 IGNITE-19276 Implement a mechanism to build indices 
distributively (#2676)
85f4fe5564 is described below

commit 85f4fe5564643723adffbfa5e19fb0e84d1c4a22
Author: Kirill Tkalenko 
AuthorDate: Mon Oct 16 12:01:22 2023 +0300

IGNITE-19276 Implement a mechanism to build indices distributively (#2676)
---
 modules/index/build.gradle |   1 +
 .../index/IndexAvailabilityController.java | 361 ++
 .../index/IndexAvailabilityControllerTest.java | 403 +
 .../ignite/internal/table/TableTestUtils.java  |  31 +-
 4 files changed, 792 insertions(+), 4 deletions(-)

diff --git a/modules/index/build.gradle b/modules/index/build.gradle
index 264ef39637..367f7661c4 100644
--- a/modules/index/build.gradle
+++ b/modules/index/build.gradle
@@ -41,6 +41,7 @@ dependencies {
 testImplementation(testFixtures(project(':ignite-table')))
 testImplementation(testFixtures(project(':ignite-catalog')))
 testImplementation project(':ignite-placement-driver')
+testImplementation project(':ignite-replicator')
 testImplementation libs.mockito.core
 testImplementation libs.mockito.junit
 testImplementation libs.hamcrest.core
diff --git 
a/modules/index/src/main/java/org/apache/ignite/internal/index/IndexAvailabilityController.java
 
b/modules/index/src/main/java/org/apache/ignite/internal/index/IndexAvailabilityController.java
new file mode 100644
index 00..633d0cf133
--- /dev/null
+++ 
b/modules/index/src/main/java/org/apache/ignite/internal/index/IndexAvailabilityController.java
@@ -0,0 +1,361 @@
+/*
+ * 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.index;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static java.util.concurrent.CompletableFuture.completedFuture;
+import static java.util.concurrent.CompletableFuture.failedFuture;
+import static java.util.function.Predicate.not;
+import static java.util.stream.Collectors.toList;
+import static 
org.apache.ignite.internal.catalog.CatalogService.DEFAULT_SCHEMA_NAME;
+import static org.apache.ignite.internal.metastorage.dsl.Conditions.exists;
+import static org.apache.ignite.internal.metastorage.dsl.Conditions.notExists;
+import static org.apache.ignite.internal.metastorage.dsl.Operations.noop;
+import static org.apache.ignite.internal.metastorage.dsl.Operations.put;
+import static org.apache.ignite.internal.metastorage.dsl.Operations.remove;
+import static org.apache.ignite.internal.util.ArrayUtils.BYTE_EMPTY_ARRAY;
+import static org.apache.ignite.internal.util.CollectionUtils.concat;
+import static org.apache.ignite.internal.util.ExceptionUtils.unwrapCause;
+import static org.apache.ignite.internal.util.IgniteUtils.inBusyLock;
+import static org.apache.ignite.internal.util.IgniteUtils.inBusyLockAsync;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.stream.IntStream;
+import org.apache.ignite.internal.catalog.CatalogCommand;
+import org.apache.ignite.internal.catalog.CatalogManager;
+import org.apache.ignite.internal.catalog.CatalogService;
+import 
org.apache.ignite.internal.catalog.IndexAlreadyAvailableValidationException;
+import org.apache.ignite.internal.catalog.IndexNotFoundValidationException;
+import org.apache.ignite.internal.catalog.commands.MakeIndexAvailableCommand;
+import org.apache.ignite.internal.catalog.descriptors.CatalogIndexDescriptor;
+import org.apache.ignite.internal.catalog.descriptors.CatalogTableDescriptor;
+import org.apache.ignite.internal.catalog.descriptors.CatalogZoneDescriptor;
+import org.apache.ignite.internal.catalog.events.CatalogEvent;
+import org.apache.ignite.internal.catalog.events.CreateIndexEventParameters;
+import org.apache.ignite.internal.catalog.events.DropIndexEventParameters;
+import 
org.apache.ignite.internal.catalog.events.MakeIndexAvailableE

[ignite-3] branch main updated: IGNITE-20317 Return metastorage invokes for zones changes in handlers, immediately recalculate data nodes when scale up/down is immediate. (#2685)

2023-10-16 Thread sk0x50
This is an automated email from the ASF dual-hosted git repository.

sk0x50 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 d5d8dde793 IGNITE-20317 Return metastorage invokes for zones changes 
in handlers, immediately recalculate data nodes when scale up/down is 
immediate. (#2685)
d5d8dde793 is described below

commit d5d8dde7939c66637be66e6b62966fc9429de068
Author: Mirza Aliev 
AuthorDate: Mon Oct 16 11:11:01 2023 +0400

IGNITE-20317 Return metastorage invokes for zones changes in handlers, 
immediately recalculate data nodes when scale up/down is immediate. (#2685)
---
 .../distributionzones/DistributionZoneManager.java | 144 -
 .../rebalance/DistributionZoneRebalanceEngine.java | 128 +-
 .../DistributionZoneCausalityDataNodesTest.java|   2 -
 3 files changed, 149 insertions(+), 125 deletions(-)

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 a65a13eb16..ae1e59ef3b 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
@@ -24,6 +24,7 @@ import static 
java.util.concurrent.CompletableFuture.failedFuture;
 import static java.util.concurrent.TimeUnit.SECONDS;
 import static java.util.stream.Collectors.toList;
 import static java.util.stream.Collectors.toSet;
+import static 
org.apache.ignite.internal.catalog.commands.CatalogUtils.IMMEDIATE_TIMER_VALUE;
 import static 
org.apache.ignite.internal.catalog.commands.CatalogUtils.INFINITE_TIMER_VALUE;
 import static 
org.apache.ignite.internal.catalog.events.CatalogEvent.ZONE_ALTER;
 import static 
org.apache.ignite.internal.catalog.events.CatalogEvent.ZONE_CREATE;
@@ -150,17 +151,17 @@ public class DistributionZoneManager implements 
IgniteComponent {
 private final LogicalTopologyEventListener topologyEventListener = new 
LogicalTopologyEventListener() {
 @Override
 public void onNodeJoined(LogicalNode joinedNode, 
LogicalTopologySnapshot newTopology) {
-updateLogicalTopologyInMetaStorage(newTopology, false);
+updateLogicalTopologyInMetaStorage(newTopology);
 }
 
 @Override
 public void onNodeLeft(LogicalNode leftNode, LogicalTopologySnapshot 
newTopology) {
-updateLogicalTopologyInMetaStorage(newTopology, false);
+updateLogicalTopologyInMetaStorage(newTopology);
 }
 
 @Override
 public void onTopologyLeap(LogicalTopologySnapshot newTopology) {
-updateLogicalTopologyInMetaStorage(newTopology, true);
+updateLogicalTopologyInMetaStorage(newTopology);
 }
 };
 
@@ -295,6 +296,14 @@ public class DistributionZoneManager implements 
IgniteComponent {
 
 long causalityToken = parameters.causalityToken();
 
+if (newScaleUp == IMMEDIATE_TIMER_VALUE) {
+return saveDataNodesToMetaStorageOnScaleUp(zoneId, 
causalityToken).thenRun(() -> {
+// TODO: causalityOnUpdateScaleUp will be removed 
https://issues.apache.org/jira/browse/IGNITE-20604,
+// catalog must be used instead
+
causalityDataNodesEngine.causalityOnUpdateScaleUp(causalityToken, zoneId, 
IMMEDIATE_TIMER_VALUE);
+});
+}
+
 // It is safe to zonesTimers.get(zoneId) in term of NPE because meta 
storage notifications are one-threaded
 // and this map will be initialized on a manager start or with catalog 
notification
 ZoneState zoneState = zonesState.get(zoneId);
@@ -329,6 +338,14 @@ public class DistributionZoneManager implements 
IgniteComponent {
 
 long causalityToken = parameters.causalityToken();
 
+if (newScaleDown == IMMEDIATE_TIMER_VALUE) {
+return saveDataNodesToMetaStorageOnScaleDown(zoneId, 
causalityToken).thenRun(() -> {
+// TODO: causalityOnUpdateScaleDown will be removed 
https://issues.apache.org/jira/browse/IGNITE-20604,
+// catalog must be used instead
+
causalityDataNodesEngine.causalityOnUpdateScaleDown(causalityToken, zoneId, 
IMMEDIATE_TIMER_VALUE);
+});
+}
+
 // It is safe to zonesTimers.get(zoneId) in term of NPE because meta 
storage notifications are one-threaded
 // and this map will be initialized on a manager start or with catalog 
notification
 ZoneState zoneState = zonesState.get(zoneId);
@@ -387,8 +404,9 @@ public class DistributionZoneManager implements 
IgniteComponent {
  *
  * @param zone Zone descriptor.
  * @param causal