You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@ignite.apache.org by "Alexei Scherbakov (JIRA)" <ji...@apache.org> on 2017/09/05 13:53:00 UTC

[jira] [Commented] (IGNITE-6181) Tx is not rolled back on timeout leading to potential whole grid hang

    [ https://issues.apache.org/jira/browse/IGNITE-6181?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16153670#comment-16153670 ] 

Alexei Scherbakov commented on IGNITE-6181:
-------------------------------------------

Sam, thanks for review.

1. Removed.
2. Missed what. Fixed.
3. Removed. Probably better leave it to application side.
4. Because timeouts < 100ms have no sense and to prevent races between tx init and concurrent tx timeout
5. Consider a scenario:
Thread started a tx with a timeout.
Tx execution reached the timeout point and tx was removed by timeout handler.
Tx execution was continued
Expected behavior: all subsequent operation within current transactions must fail, otherwise they will execute as implicit transaction(new tx will start, because context is empty), which is incorrect.
To fix it I delayed threadMap's cleanup  for not loosing state on timeout until calling tx close method, which in it's turn calls onLocalClose and removes map entry completely.
6. This is needed because threadMap may contain uncleared entry for timed out transaction, as described in 5. In such case a creation of new explicit transaction is allowed.
7. I removed block for checking timeout state, because it's no longer needed. A transaction will be rolled back by timeout handler.
8, 9, 10 Fixed


> Tx is not rolled back on timeout leading to potential whole grid hang
> ---------------------------------------------------------------------
>
>                 Key: IGNITE-6181
>                 URL: https://issues.apache.org/jira/browse/IGNITE-6181
>             Project: Ignite
>          Issue Type: Improvement
>    Affects Versions: 2.1
>            Reporter: Alexei Scherbakov
>            Assignee: Alexei Scherbakov
>             Fix For: 2.3
>
>
> Unit test reproducer:
> {noformat}
> /*
>  * 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.cache;
> import java.util.concurrent.CountDownLatch;
> import java.util.concurrent.TimeUnit;
> import java.util.concurrent.atomic.AtomicBoolean;
> import org.apache.ignite.Ignite;
> import org.apache.ignite.IgniteException;
> import org.apache.ignite.configuration.CacheConfiguration;
> import org.apache.ignite.configuration.IgniteConfiguration;
> import org.apache.ignite.configuration.TransactionConfiguration;
> import org.apache.ignite.internal.IgniteInternalFuture;
> import org.apache.ignite.internal.util.typedef.internal.U;
> import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
> import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
> import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
> import org.apache.ignite.transactions.Transaction;
> import org.apache.ignite.transactions.TransactionConcurrency;
> import org.apache.ignite.transactions.TransactionIsolation;
> /**
>  * Tests ability to rollback not properly closed transaction.
>  */
> public class IgniteTxTimeoutTest extends GridCommonAbstractTest {
>     /** */
>     private static final long TX_TIMEOUT = 3_000L;
>     /** */
>     private static final String CACHE_NAME = "test";
>     /** IP finder. */
>     private static final TcpDiscoveryVmIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
>     /** */
>     private final CountDownLatch l = new CountDownLatch(1);
>     /** */
>     private final Object mux = new Object();
>     /** {@inheritDoc} */
>     @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
>         IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
>         cfg.setDiscoverySpi(new TcpDiscoverySpi().setIpFinder(IP_FINDER));
>         TransactionConfiguration txCfg = new TransactionConfiguration();
>         txCfg.setDefaultTxTimeout(TX_TIMEOUT);
>         cfg.setTransactionConfiguration(txCfg);
>         CacheConfiguration ccfg = new CacheConfiguration(CACHE_NAME);
>         ccfg.setAtomicityMode(CacheAtomicityMode.TRANSACTIONAL);
>         cfg.setCacheConfiguration(ccfg);
>         return cfg;
>     }
>     /** */
>     public void testTxTimeoutHandling() throws Exception {
>         try {
>             final Ignite ignite = startGrid(0);
>             final AtomicBoolean released = new AtomicBoolean();
>             multithreadedAsync(new Runnable() {
>                 @Override public void run() {
>                     // Start tx with default settings.
>                     try (Transaction tx = ignite.transactions().txStart()) {
>                         ignite.cache(CACHE_NAME).put(1, 1);
>                         l.countDown();
>                         // Wait longer than default timeout.
>                         synchronized (mux) {
>                             while (!released.get()) {
>                                 try {
>                                     mux.wait();
>                                 }
>                                 catch (InterruptedException e) {
>                                     throw new IgniteException(e);
>                                 }
>                             }
>                         }
>                         try {
>                             tx.commit();
>                             fail();
>                         }
>                         catch (IgniteException e) {
>                             // Expect exception - tx is rolled back.
>                         }
>                     }
>                 }
>             }, 1, "Locker");
>             IgniteInternalFuture<?> fut2 = multithreadedAsync(new Runnable() {
>                 @Override public void run() {
>                     U.awaitQuiet(l);
>                     // Try to acquire lock.
>                     // Acquisition will be successul then first transaction will be rolled back after timeout.
>                     try (Transaction tx = ignite.transactions().txStart(TransactionConcurrency.PESSIMISTIC,
>                         TransactionIsolation.REPEATABLE_READ, 0, 1)) {
>                         ignite.cache(CACHE_NAME).put(1, 1);
>                         tx.commit();
>                     }
>                 }
>             }, 1, "Waiter");
>             Thread.sleep(TX_TIMEOUT + 1_000);
>             fut2.get(1, TimeUnit.SECONDS);
>             
>             startGrid(1);
>             
>             released.set(true);
>             synchronized (mux) {
>                 mux.notify();
>             }
>         }
>         finally {
>             stopAllGrids();
>         }
>     }
> }
> {noformat}



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)