You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@ignite.apache.org by GitBox <gi...@apache.org> on 2020/07/07 16:21:57 UTC

[GitHub] [ignite] ptupitsyn commented on a change in pull request #7992: IGNITE-7369 Thin Client Transactions [DRAFT]

ptupitsyn commented on a change in pull request #7992:
URL: https://github.com/apache/ignite/pull/7992#discussion_r450973219



##########
File path: modules/platforms/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj
##########
@@ -143,6 +143,9 @@
     <Compile Include="Cache\Query\Linq\CacheLinqTest.Contains.cs" />
     <Compile Include="Cache\Store\CacheStoreSessionTestCodeConfig.cs" />
     <Compile Include="Cache\Store\CacheStoreSessionTestSharedFactory.cs" />
+    <Compile Include="Client\Cache\CacheClientAbstractTxTest.cs" />

Review comment:
       Please add those tests to `.DotNetCore` project as well.

##########
File path: modules/platforms/dotnet/Apache.Ignite.Core.Tests/Client/Cache/CacheClientLocalTxTest.cs
##########
@@ -0,0 +1,38 @@
+/*
+ * 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.
+ */
+
+namespace Apache.Ignite.Core.Tests.Client.Cache
+{
+    /// <summary>
+    /// Tests client transactions for single node.
+    /// </summary>
+    public class CacheClientLocalTxTest : CacheClientAbstractTxTest
+    {
+        /// <summary>
+        ///  Initializes a new instance of the <see cref="CacheClientLocalTxTest"/> class.
+        /// </summary>
+        public CacheClientLocalTxTest() : base(1, false)
+        {
+            // No-op.
+        }
+
+        protected override string GetCacheName()

Review comment:
       Missing `inheritdoc`

##########
File path: modules/platforms/dotnet/Apache.Ignite.Core/Client/Transactions/IClientTransactions.cs
##########
@@ -0,0 +1,58 @@
+/*
+ * 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.
+ */
+
+namespace Apache.Ignite.Core.Client.Transactions
+{
+    using System;
+    using Apache.Ignite.Core.Transactions;
+
+    /// <summary>
+    /// Ignite Thin Client transactions facade.
+    /// </summary>
+    public interface IClientTransactions
+    {
+        /// <summary>
+        /// Starts a new transaction with the default isolation level, concurrency and timeout. 
+        /// </summary>
+        /// <returns>New transaction.</returns>
+        IClientTransaction TxStart();
+
+        /// <summary>
+        /// Starts new transaction with the specified concurrency and isolation.
+        /// </summary>
+        /// <param name="concurrency">Concurrency.</param>
+        /// <param name="isolation">Isolation.</param>
+        /// <returns>New transaction.</returns>
+        IClientTransaction TxStart(TransactionConcurrency concurrency, TransactionIsolation isolation);
+
+        /// <summary>
+        /// Starts new transaction with the specified concurrency, isolation and timeout.
+        /// </summary>
+        /// <param name="concurrency">Concurrency.</param>
+        /// <param name="isolation">Isolation.</param>
+        /// <param name="timeout">Timeout. TimeSpan. Zero for indefinite timeout.</param>
+        /// <returns>New transaction.</returns>
+        IClientTransaction TxStart(TransactionConcurrency concurrency, TransactionIsolation isolation, TimeSpan timeout);
+
+        /// <summary>
+        /// Returns instance of <see cref="IClientTransactions"/>> to mark a transaction with a special label.

Review comment:
       Please add more explanation - why is this useful, how to retrieve this label

##########
File path: modules/platforms/dotnet/Apache.Ignite.Core/Client/Transactions/IClientTransactions.cs
##########
@@ -0,0 +1,58 @@
+/*
+ * 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.
+ */
+
+namespace Apache.Ignite.Core.Client.Transactions
+{
+    using System;
+    using Apache.Ignite.Core.Transactions;
+
+    /// <summary>
+    /// Ignite Thin Client transactions facade.
+    /// </summary>
+    public interface IClientTransactions

Review comment:
       Let's rename to `ITransactionsClient` to be consistent with `ICacheClient`, `IComputeClient`.

##########
File path: modules/platforms/dotnet/Apache.Ignite.Core/Client/Transactions/IClientTransaction.cs
##########
@@ -0,0 +1,37 @@
+/*
+ * 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.
+ */
+
+namespace Apache.Ignite.Core.Client.Transactions
+{
+    using System;
+
+    /// <summary>
+    /// Thin client transaction.
+    /// </summary>
+    public interface IClientTransaction : IDisposable

Review comment:
       Let's rename to `ITransactionClient` to be consistent with `ICacheClient`, `IComputeClient`.

##########
File path: modules/platforms/dotnet/Apache.Ignite.Core/Impl/Client/Cache/CacheClient.cs
##########
@@ -754,14 +797,26 @@ private void WriteRequest(Action<ClientRequestContext> writeAction, ClientReques
         {
             ctx.Stream.WriteInt(_id);
 
+            var flags = ClientCacheRequestFlag.None;
             if (_expiryPolicy != null)
             {
                 ctx.Features.ValidateWithExpiryPolicyFlag();
-                ctx.Stream.WriteByte((byte) ClientCacheRequestFlag.WithExpiryPolicy);
-                ExpiryPolicySerializer.WritePolicy(ctx.Writer, _expiryPolicy);
+                flags = flags | ClientCacheRequestFlag.WithExpiryPolicy;
             }
-            else
-                ctx.Stream.WriteByte((byte) ClientCacheRequestFlag.None); // Flags (skipStore, etc).
+
+            var tx = _ignite.Transactions.CurrentTx;
+            if (tx != null)
+            {
+                flags |= ClientCacheRequestFlag.WithTransactional;
+            }
+
+            ctx.Stream.WriteByte((byte) flags);
+
+            if (flags.HasFlag(ClientCacheRequestFlag.WithExpiryPolicy))

Review comment:
       `HasFlag` is [slow on older frameworks](https://www.code4it.dev/blog/hasflag-performance-benchmarkdotnet), let's avoid it here.

##########
File path: modules/platforms/dotnet/Apache.Ignite.Core/Client/IIgniteClient.cs
##########
@@ -111,6 +112,11 @@ public interface IIgniteClient : IDisposable
         [SuppressMessage("Microsoft.Design", "CA1024:UsePropertiesWhereAppropriate", Justification = "Semantics.")]
         IBinary GetBinary();
 
+        /// <summary>
+        /// Gets Ignite transactions facade.
+        /// </summary>
+        IClientTransactions Transactions { get; }

Review comment:
       Please convert to a method for consistency with `IIgnite.GetTransactions`.
   Method is also a more future-proof option (see https://docs.microsoft.com/en-us/previous-versions/dotnet/netframework-4.0/ms229054(v=vs.100)).

##########
File path: modules/platforms/dotnet/Apache.Ignite.Core/Impl/Client/Transactions/ClientTransaction.cs
##########
@@ -0,0 +1,125 @@
+/*
+ * 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.
+ */
+
+namespace Apache.Ignite.Core.Impl.Client.Transactions
+{
+    using System;
+    using System.Threading;
+
+    /// <summary>
+    /// Ignite Thin Client transaction facade.
+    /// </summary>
+    internal class ClientTransaction: IClientTransactionInternal
+    {
+        /** Unique  transaction ID.*/
+        private readonly int _id;
+
+        /** Ignite. */
+        private readonly IgniteClient _ignite;
+
+        /* Transactions. */
+        private readonly ClientTransactions _transactions;
+
+        /** Transaction is closed. */
+        private volatile bool _closed; 

Review comment:
       Do we need `volatile`? Transactions are tied to a specific thread, right?

##########
File path: modules/platforms/dotnet/Apache.Ignite.Core/Impl/Client/ClientFailoverSocket.cs
##########
@@ -269,6 +278,12 @@ private ClientSocket GetAffinitySocket<TKey>(int cacheId, TKey key)
                 return null;
             }
 
+            // Transactional operation should be executed on node started the transaction.
+            if (_transactions.CurrentTx != null)

Review comment:
       To be precise: transactional operation must be executed using the same connection (same `ClientSocket` instance) that started the transaction.
   
   And the check here does not guarantee this at all - reconnect could have happened, for example.
   A proper approach is to store get the socket from `ClientContextBase.Socket` property when executing `ClientOp.TxStart`, and use that socket directly for transactional operations.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org