[ 
https://issues.apache.org/jira/browse/IGNITE-27180?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Igor Sapego updated IGNITE-27180:
---------------------------------
    Description: 
h4. Scenario:

1. Java client connected to 3 nodes
2. Run partition aware request
3. Stop one node gacefully
4. Ensure connections count = 2
5. Run partition aware request
h4. Expected:

Partition aware request completed successfully.
(but since the primary node has left, it can be sent to a random alive node).
h4. Actual:

Request fails with "Connection refused":
{noformat}
org.apache.ignite.client.IgniteClientConnectionException: IGN-CLIENT-1 Client 
failed to connect: Connection refused: /127.0.0.1:10800 
[endpoint=127.0.0.1:10800]
 TraceId:e1bfd0dd
  at 
java.base/java.lang.invoke.MethodHandle.invokeWithArguments(MethodHandle.java:710)
  at 
org.apache.ignite.internal.util.ExceptionUtils$1.copy(ExceptionUtils.java:952)
  at 
org.apache.ignite.internal.util.ExceptionUtils$ExceptionFactory.createCopy(ExceptionUtils.java:886)
  at 
org.apache.ignite.internal.util.ExceptionUtils.copyExceptionWithCause(ExceptionUtils.java:688)
  at 
org.apache.ignite.internal.util.ViewUtils.copyExceptionWithCauseIfPossible(ViewUtils.java:91)
  at 
org.apache.ignite.internal.util.ViewUtils.ensurePublicException(ViewUtils.java:71)
  at org.apache.ignite.internal.util.ViewUtils.sync(ViewUtils.java:54)
  at 
org.apache.ignite.internal.client.table.ClientKeyValueView.put(ClientKeyValueView.java:270)
  at 
org.apache.ignite.internal.client.table.api.PublicApiClientKeyValueView.lambda$put$12(PublicApiClientKeyValueView.java:113)
  at 
org.apache.ignite.internal.thread.PublicApiThreading.lambda$execUserSyncOperation$1(PublicApiThreading.java:116)
  at 
org.apache.ignite.internal.thread.PublicApiThreading.executeWithRole(PublicApiThreading.java:144)
  at 
org.apache.ignite.internal.thread.PublicApiThreading.execUserSyncOperation(PublicApiThreading.java:102)
  at 
org.apache.ignite.internal.thread.PublicApiThreading.execUserSyncOperation(PublicApiThreading.java:115)
  at 
org.apache.ignite.internal.client.table.api.PublicApiClientViewBase.executeSyncOp(PublicApiClientViewBase.java:105)
  at 
org.apache.ignite.internal.client.table.api.PublicApiClientKeyValueView.put(PublicApiClientKeyValueView.java:113)
  at 
org.apache.ignite.internal.runner.app.client.ItThinConnectionFailoverTest.testStopNodePartitionAwarenessKeyValue(ItThinConnectionFailoverTest.java:49)
{noformat}
h4. Reproducer:
{code:java}
public class ItThinConnectionFailoverTest extends ClusterPerTestIntegrationTest 
{
    @Override
    protected int initialNodes() {
        return 0;
    }

    @Test
    void testStopNodePartitionAwarenessKeyValue() {
        int nodesCount = 3;
        cluster.startAndInit(nodesCount, new int[]{2});

        String[] addresses = IntStream.range(0, nodesCount)
                .mapToObj(i -> "127.0.0.1:" + (10800 + i))
                .toArray(String[]::new);

        try (IgniteClient client = IgniteClient.builder()
                .addresses(addresses).build()) {

            client.sql().executeScript("CREATE ZONE zone1 (REPLICAS 3) STORAGE 
PROFILES ['default'];"
                    + "CREATE TABLE t(id INT PRIMARY KEY, val INT) ZONE zone1");

            Awaitility.await().until(() -> client.connections().size(), is(3));

            KeyValueView<Integer, Integer> kvView = client.tables().table("t")
                    .keyValueView(Integer.class, Integer.class);

            for (int i = 0; i < 100; i++) {
                kvView.put(null, i, i);
            }

            cluster.stopNode(0);
            assertThat(client.connections().size(), is(2));

            for (int i = 100; i < 200; i++) {
                kvView.put(null, i, i); // fails with "Connection refused"
            }
        }
    }
}
{code}
Note: this also affects SQL.please check related TODOs in the code (if exist)

  was:
h4. Scenario:

1. Thin client connected to 3 nodes
2. Run partition aware request
3. Stop one node gacefully
4. Ensure connections count = 2
5. Run partition aware request

h4. Expected:

Partition aware request completed successfully.
(but since the primary node has left, it can be sent to a random alive node).

h4. Actual:

Request fails with "Connection refused":

{noformat}
org.apache.ignite.client.IgniteClientConnectionException: IGN-CLIENT-1 Client 
failed to connect: Connection refused: /127.0.0.1:10800 
[endpoint=127.0.0.1:10800]
 TraceId:e1bfd0dd
  at 
java.base/java.lang.invoke.MethodHandle.invokeWithArguments(MethodHandle.java:710)
  at 
org.apache.ignite.internal.util.ExceptionUtils$1.copy(ExceptionUtils.java:952)
  at 
org.apache.ignite.internal.util.ExceptionUtils$ExceptionFactory.createCopy(ExceptionUtils.java:886)
  at 
org.apache.ignite.internal.util.ExceptionUtils.copyExceptionWithCause(ExceptionUtils.java:688)
  at 
org.apache.ignite.internal.util.ViewUtils.copyExceptionWithCauseIfPossible(ViewUtils.java:91)
  at 
org.apache.ignite.internal.util.ViewUtils.ensurePublicException(ViewUtils.java:71)
  at org.apache.ignite.internal.util.ViewUtils.sync(ViewUtils.java:54)
  at 
org.apache.ignite.internal.client.table.ClientKeyValueView.put(ClientKeyValueView.java:270)
  at 
org.apache.ignite.internal.client.table.api.PublicApiClientKeyValueView.lambda$put$12(PublicApiClientKeyValueView.java:113)
  at 
org.apache.ignite.internal.thread.PublicApiThreading.lambda$execUserSyncOperation$1(PublicApiThreading.java:116)
  at 
org.apache.ignite.internal.thread.PublicApiThreading.executeWithRole(PublicApiThreading.java:144)
  at 
org.apache.ignite.internal.thread.PublicApiThreading.execUserSyncOperation(PublicApiThreading.java:102)
  at 
org.apache.ignite.internal.thread.PublicApiThreading.execUserSyncOperation(PublicApiThreading.java:115)
  at 
org.apache.ignite.internal.client.table.api.PublicApiClientViewBase.executeSyncOp(PublicApiClientViewBase.java:105)
  at 
org.apache.ignite.internal.client.table.api.PublicApiClientKeyValueView.put(PublicApiClientKeyValueView.java:113)
  at 
org.apache.ignite.internal.runner.app.client.ItThinConnectionFailoverTest.testStopNodePartitionAwarenessKeyValue(ItThinConnectionFailoverTest.java:49)
{noformat}

h4. Reproducer:

{code:java}
public class ItThinConnectionFailoverTest extends ClusterPerTestIntegrationTest 
{
    @Override
    protected int initialNodes() {
        return 0;
    }

    @Test
    void testStopNodePartitionAwarenessKeyValue() {
        int nodesCount = 3;
        cluster.startAndInit(nodesCount, new int[]{2});

        String[] addresses = IntStream.range(0, nodesCount)
                .mapToObj(i -> "127.0.0.1:" + (10800 + i))
                .toArray(String[]::new);

        try (IgniteClient client = IgniteClient.builder()
                .addresses(addresses).build()) {

            client.sql().executeScript("CREATE ZONE zone1 (REPLICAS 3) STORAGE 
PROFILES ['default'];"
                    + "CREATE TABLE t(id INT PRIMARY KEY, val INT) ZONE zone1");

            Awaitility.await().until(() -> client.connections().size(), is(3));

            KeyValueView<Integer, Integer> kvView = client.tables().table("t")
                    .keyValueView(Integer.class, Integer.class);

            for (int i = 0; i < 100; i++) {
                kvView.put(null, i, i);
            }

            cluster.stopNode(0);
            assertThat(client.connections().size(), is(2));

            for (int i = 100; i < 200; i++) {
                kvView.put(null, i, i); // fails with "Connection refused"
            }
        }
    }
}
{code}

Note: this also affects SQL.please check related TODOs in the code (if exist)


> Java Client: Partition awareness request fails with "connection refused" 
> after node shutdown gracefully
> -------------------------------------------------------------------------------------------------------
>
>                 Key: IGNITE-27180
>                 URL: https://issues.apache.org/jira/browse/IGNITE-27180
>             Project: Ignite
>          Issue Type: Bug
>          Components: platforms ai3, thin clients ai3
>            Reporter: Pavel Pereslegin
>            Assignee: Pavel Tupitsyn
>            Priority: Major
>              Labels: ignite-3
>         Attachments: ItThinConnectionFailoverTest.java
>
>
> h4. Scenario:
> 1. Java client connected to 3 nodes
> 2. Run partition aware request
> 3. Stop one node gacefully
> 4. Ensure connections count = 2
> 5. Run partition aware request
> h4. Expected:
> Partition aware request completed successfully.
> (but since the primary node has left, it can be sent to a random alive node).
> h4. Actual:
> Request fails with "Connection refused":
> {noformat}
> org.apache.ignite.client.IgniteClientConnectionException: IGN-CLIENT-1 Client 
> failed to connect: Connection refused: /127.0.0.1:10800 
> [endpoint=127.0.0.1:10800]
>  TraceId:e1bfd0dd
>   at 
> java.base/java.lang.invoke.MethodHandle.invokeWithArguments(MethodHandle.java:710)
>   at 
> org.apache.ignite.internal.util.ExceptionUtils$1.copy(ExceptionUtils.java:952)
>   at 
> org.apache.ignite.internal.util.ExceptionUtils$ExceptionFactory.createCopy(ExceptionUtils.java:886)
>   at 
> org.apache.ignite.internal.util.ExceptionUtils.copyExceptionWithCause(ExceptionUtils.java:688)
>   at 
> org.apache.ignite.internal.util.ViewUtils.copyExceptionWithCauseIfPossible(ViewUtils.java:91)
>   at 
> org.apache.ignite.internal.util.ViewUtils.ensurePublicException(ViewUtils.java:71)
>   at org.apache.ignite.internal.util.ViewUtils.sync(ViewUtils.java:54)
>   at 
> org.apache.ignite.internal.client.table.ClientKeyValueView.put(ClientKeyValueView.java:270)
>   at 
> org.apache.ignite.internal.client.table.api.PublicApiClientKeyValueView.lambda$put$12(PublicApiClientKeyValueView.java:113)
>   at 
> org.apache.ignite.internal.thread.PublicApiThreading.lambda$execUserSyncOperation$1(PublicApiThreading.java:116)
>   at 
> org.apache.ignite.internal.thread.PublicApiThreading.executeWithRole(PublicApiThreading.java:144)
>   at 
> org.apache.ignite.internal.thread.PublicApiThreading.execUserSyncOperation(PublicApiThreading.java:102)
>   at 
> org.apache.ignite.internal.thread.PublicApiThreading.execUserSyncOperation(PublicApiThreading.java:115)
>   at 
> org.apache.ignite.internal.client.table.api.PublicApiClientViewBase.executeSyncOp(PublicApiClientViewBase.java:105)
>   at 
> org.apache.ignite.internal.client.table.api.PublicApiClientKeyValueView.put(PublicApiClientKeyValueView.java:113)
>   at 
> org.apache.ignite.internal.runner.app.client.ItThinConnectionFailoverTest.testStopNodePartitionAwarenessKeyValue(ItThinConnectionFailoverTest.java:49)
> {noformat}
> h4. Reproducer:
> {code:java}
> public class ItThinConnectionFailoverTest extends 
> ClusterPerTestIntegrationTest {
>     @Override
>     protected int initialNodes() {
>         return 0;
>     }
>     @Test
>     void testStopNodePartitionAwarenessKeyValue() {
>         int nodesCount = 3;
>         cluster.startAndInit(nodesCount, new int[]{2});
>         String[] addresses = IntStream.range(0, nodesCount)
>                 .mapToObj(i -> "127.0.0.1:" + (10800 + i))
>                 .toArray(String[]::new);
>         try (IgniteClient client = IgniteClient.builder()
>                 .addresses(addresses).build()) {
>             client.sql().executeScript("CREATE ZONE zone1 (REPLICAS 3) 
> STORAGE PROFILES ['default'];"
>                     + "CREATE TABLE t(id INT PRIMARY KEY, val INT) ZONE 
> zone1");
>             Awaitility.await().until(() -> client.connections().size(), 
> is(3));
>             KeyValueView<Integer, Integer> kvView = client.tables().table("t")
>                     .keyValueView(Integer.class, Integer.class);
>             for (int i = 0; i < 100; i++) {
>                 kvView.put(null, i, i);
>             }
>             cluster.stopNode(0);
>             assertThat(client.connections().size(), is(2));
>             for (int i = 100; i < 200; i++) {
>                 kvView.put(null, i, i); // fails with "Connection refused"
>             }
>         }
>     }
> }
> {code}
> Note: this also affects SQL.please check related TODOs in the code (if exist)



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

Reply via email to