You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by pt...@apache.org on 2020/06/09 09:28:45 UTC

[ignite] branch master updated: IGNITE-7105 .NET: Add IgniteLock

This is an automated email from the ASF dual-hosted git repository.

ptupitsyn 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 2026b25  IGNITE-7105 .NET: Add IgniteLock
2026b25 is described below

commit 2026b25d87c1d01dce8e33b2dc3d6ae5b3af9df5
Author: Pavel Tupitsyn <pt...@apache.org>
AuthorDate: Tue Jun 9 12:28:31 2020 +0300

    IGNITE-7105 .NET: Add IgniteLock
    
    * Add `IIgnite.GetOrCreateLock` (no `reentrant` because .NET locks are reentrant by default, this is expected)
    * `IIgniteLock` methods are named according to the standard `Monitor` class and existing Ignite `ICacheLock`
---
 .../internal/processors/platform/PlatformLock.java | 108 +++++++
 .../processors/platform/PlatformProcessorImpl.java |  15 +
 .../datastructures/IgniteLockAbstractSelfTest.java |  20 ++
 .../Apache.Ignite.Core.Tests.DotNetCore.csproj     |   6 +
 .../Apache.Ignite.Core.Tests.csproj                |   2 +
 .../IgniteLockFailoverTests.cs                     | 105 +++++++
 .../Apache.Ignite.Core.Tests/IgniteLockTests.cs    | 329 +++++++++++++++++++++
 .../dotnet/Apache.Ignite.Core.Tests/TestBase.cs    |  31 +-
 .../Apache.Ignite.Core/Apache.Ignite.Core.csproj   |   3 +
 .../Configuration/LockConfiguration.cs             |  68 +++++
 .../platforms/dotnet/Apache.Ignite.Core/IIgnite.cs |  33 ++-
 .../dotnet/Apache.Ignite.Core/IIgniteLock.cs       |  81 +++++
 .../Apache.Ignite.Core/Impl/ExceptionUtils.cs      |   5 +-
 .../dotnet/Apache.Ignite.Core/Impl/Ignite.cs       |  37 ++-
 .../dotnet/Apache.Ignite.Core/Impl/IgniteLock.cs   | 114 +++++++
 .../Apache.Ignite.DotNetCore.sln.DotSettings       |   2 +
 16 files changed, 946 insertions(+), 13 deletions(-)

diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformLock.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformLock.java
new file mode 100644
index 0000000..3b529bd
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformLock.java
@@ -0,0 +1,108 @@
+/*
+ * 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.processors.platform;
+
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteLock;
+
+import java.util.concurrent.TimeUnit;
+
+/**
+ * Platform wrapper for {@link IgniteLock}.
+ */
+class PlatformLock extends PlatformAbstractTarget {
+    /** */
+    private static final int OP_LOCK = 1;
+
+    /** */
+    private static final int OP_TRY_LOCK = 2;
+
+    /** */
+    private static final int OP_UNLOCK = 3;
+
+    /** */
+    private static final int OP_REMOVE = 4;
+
+    /** */
+    private static final int OP_IS_BROKEN = 5;
+
+    /** */
+    private static final int OP_IS_LOCKED = 6;
+
+    /** */
+    private static final int OP_IS_REMOVED = 7;
+
+    /** Wrapped lock. */
+    private final IgniteLock lock;
+
+    /**
+     * Constructor.
+     * @param ctx Context.
+     * @param lock Lock to wrap.
+     */
+    PlatformLock(PlatformContext ctx, IgniteLock lock) {
+        super(ctx);
+
+        this.lock = lock;
+    }
+
+    /** {@inheritDoc} */
+    @Override public long processInLongOutLong(int type, long val) throws IgniteCheckedException {
+        switch (type) {
+            case OP_LOCK: {
+                lock.lock();
+
+                return TRUE;
+            }
+
+            case OP_TRY_LOCK: {
+                boolean locked = val < 0
+                        ? lock.tryLock()
+                        : lock.tryLock(val, TimeUnit.MILLISECONDS);
+
+                return locked ? TRUE : FALSE;
+            }
+
+            case OP_UNLOCK: {
+                lock.unlock();
+
+                return TRUE;
+            }
+
+            case OP_REMOVE: {
+                lock.close();
+
+                return TRUE;
+            }
+
+            case OP_IS_BROKEN: {
+                return lock.isBroken() ? TRUE : FALSE;
+            }
+
+            case OP_IS_LOCKED: {
+                return lock.isLocked() ? TRUE : FALSE;
+            }
+
+            case OP_IS_REMOVED: {
+                return lock.removed() ? TRUE : FALSE;
+            }
+        }
+
+        return super.processInLongOutLong(type, val);
+    }
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformProcessorImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformProcessorImpl.java
index 5380502..1117a60 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformProcessorImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformProcessorImpl.java
@@ -22,6 +22,7 @@ import org.apache.ignite.IgniteAtomicSequence;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteDataStreamer;
 import org.apache.ignite.IgniteException;
+import org.apache.ignite.IgniteLock;
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.cluster.BaselineNode;
 import org.apache.ignite.configuration.CacheConfiguration;
@@ -190,6 +191,9 @@ public class PlatformProcessorImpl extends GridProcessorAdapter implements Platf
     /** */
     private static final int OP_GET_THREAD_LOCAL = 37;
 
+    /** */
+    private static final int OP_GET_OR_CREATE_LOCK = 38;
+
     /** Start latch. */
     private final CountDownLatch startLatch = new CountDownLatch(1);
 
@@ -725,6 +729,17 @@ public class PlatformProcessorImpl extends GridProcessorAdapter implements Platf
 
                 return new PlatformTransactions(platformCtx, lbl);
             }
+
+            case OP_GET_OR_CREATE_LOCK: {
+                String name = reader.readString();
+                boolean failoverSafe = reader.readBoolean();
+                boolean fair = reader.readBoolean();
+                boolean create = reader.readBoolean();
+
+                IgniteLock lock = ctx.grid().reentrantLock(name, failoverSafe, fair, create);
+
+                return lock == null ? null : new PlatformLock(platformCtx, lock);
+            }
         }
 
         return PlatformAbstractTarget.throwUnsupported(type);
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/IgniteLockAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/IgniteLockAbstractSelfTest.java
index 5ebfa15..5d3ed99 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/IgniteLockAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/IgniteLockAbstractSelfTest.java
@@ -57,6 +57,7 @@ import org.apache.ignite.resources.LoggerResource;
 import org.apache.ignite.testframework.GridTestUtils;
 import org.apache.ignite.transactions.Transaction;
 import org.jetbrains.annotations.Nullable;
+import org.junit.Ignore;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.ExpectedException;
@@ -1680,6 +1681,25 @@ public abstract class IgniteLockAbstractSelfTest extends IgniteAtomicsAbstractTe
         ignite.close();
     }
 
+    /**
+     * Tests that closed lock throws meaningful exception.
+     */
+    @Test
+    @Ignore("https://issues.apache.org/jira/browse/IGNITE-13128")
+    public void testClosedLockThrowsIgniteException() {
+        final String lockName = "testRemovedLockThrowsIgniteException";
+
+        Ignite srv = ignite(0);
+
+        IgniteLock lock1 = srv.reentrantLock(lockName, false, false, true);
+        IgniteLock lock2 = srv.reentrantLock(lockName, false, false, true);
+
+        lock1.close();
+
+        //noinspection ThrowableNotThrown
+        GridTestUtils.assertThrows(log, lock2::lock, IgniteException.class, "TODO");
+    }
+
     /** {@inheritDoc} */
     @Override public void writeExternal(ObjectOutput out) throws IOException {
         // No-op.
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests.DotNetCore/Apache.Ignite.Core.Tests.DotNetCore.csproj b/modules/platforms/dotnet/Apache.Ignite.Core.Tests.DotNetCore/Apache.Ignite.Core.Tests.DotNetCore.csproj
index 0178a0c..cb868a1 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests.DotNetCore/Apache.Ignite.Core.Tests.DotNetCore.csproj
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests.DotNetCore/Apache.Ignite.Core.Tests.DotNetCore.csproj
@@ -285,6 +285,12 @@
     </Compile>
     <Compile Include="..\Apache.Ignite.Core.Tests\EventsTestLocalListeners.cs" Link="Common\EventsTestLocalListeners.cs" />
     <Compile Include="..\Apache.Ignite.Core.Tests\IgniteConfigurationTest.cs" Link="Common\IgniteConfigurationTest.cs" />
+    <Compile Include="..\Apache.Ignite.Core.Tests\IgniteLockFailoverTests.cs">
+      <Link>Common\IgniteLockFailoverTests.cs</Link>
+    </Compile>
+    <Compile Include="..\Apache.Ignite.Core.Tests\IgniteLockTests.cs">
+      <Link>Common\IgniteLockTests.cs</Link>
+    </Compile>
     <Compile Include="..\Apache.Ignite.Core.Tests\IgniteUtilsTest.cs">
       <Link>Common\IgniteUtilsTest.cs</Link>
     </Compile>
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj
index d32da1c..bc1aedc 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Apache.Ignite.Core.Tests.csproj
@@ -239,6 +239,8 @@
     <Compile Include="EnvVar.cs" />
     <Compile Include="EventsTestLocalListeners.cs" />
     <Compile Include="FailureHandlerTest.cs" />
+    <Compile Include="IgniteLockFailoverTests.cs" />
+    <Compile Include="IgniteLockTests.cs" />
     <Compile Include="IgniteUtilsTest.cs" />
     <Compile Include="Impl\Compute\ComputeImplTest.cs" />
     <Compile Include="JavaServer.cs" />
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteLockFailoverTests.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteLockFailoverTests.cs
new file mode 100644
index 0000000..e6ab457
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteLockFailoverTests.cs
@@ -0,0 +1,105 @@
+/*
+ * 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
+{
+    using System.Threading;
+    using System.Threading.Tasks;
+    using Apache.Ignite.Core.Common;
+    using Apache.Ignite.Core.Configuration;
+    using NUnit.Framework;
+
+    /// <summary>
+    /// Tests <see cref="IIgniteLock"/> failover.
+    /// </summary>
+    public class IgniteLockFailoverTests
+    {
+        /// <summary>
+        /// Tears down the test.
+        /// </summary>
+        [TearDown]
+        public void TearDown()
+        {
+            Ignition.StopAll(true);
+        }
+
+        /// <summary>
+        /// Tests that "lock broken" exception is thrown when lock is not failover-safe and owner node leaves.
+        /// </summary>
+        [Test]
+        public void TestNonFailoverSafeLockThrowsExceptionOnAllNodesWhenOwnerLeaves()
+        {
+            var lock1 = TestFailover(false);
+            var ex = Assert.Throws<IgniteException>(() => lock1.Enter());
+
+            StringAssert.StartsWith("Lock broken", ex.Message);
+            Assert.IsTrue(lock1.IsBroken());
+        }
+
+        /// <summary>
+        /// Tests that failover-safe lock releases when owner node leaves.
+        /// </summary>
+        [Test]
+        public void TestFailoverSafeLockIsReleasedWhenOwnerLeaves()
+        {
+            var lock1 = TestFailover(true);
+            lock1.Enter();
+
+            Assert.IsTrue(lock1.IsEntered());
+            Assert.IsFalse(lock1.IsBroken());
+        }
+
+        /// <summary>
+        /// Tests failover scenario when lock owner node leaves.
+        /// </summary>
+        private static IIgniteLock TestFailover(bool isFailoverSafe)
+        {
+            var ignite = Ignition.Start(TestUtils.GetTestConfiguration());
+
+            var cfg = new LockConfiguration
+            {
+                Name = TestUtils.TestName,
+                IsFailoverSafe = isFailoverSafe
+            };
+
+            var lock1 = ignite.GetOrCreateLock(cfg, true);
+            var evt = new ManualResetEventSlim(false);
+
+            Task.Factory.StartNew(() =>
+            {
+                var igniteCfg = new IgniteConfiguration(TestUtils.GetTestConfiguration())
+                {
+                    IgniteInstanceName = cfg.Name
+                };
+
+                var ignite2 = Ignition.Start(igniteCfg);
+
+                var lock2 = ignite2.GetOrCreateLock(cfg, true);
+                lock2.Enter();
+
+                evt.Set();
+                Thread.Sleep(100);
+
+                Ignition.Stop(cfg.Name, true);
+            });
+
+            evt.Wait();
+
+            return lock1;
+        }
+    }
+}
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteLockTests.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteLockTests.cs
new file mode 100644
index 0000000..0436952
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteLockTests.cs
@@ -0,0 +1,329 @@
+/*
+ * 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
+{
+    using System;
+    using System.Collections.Concurrent;
+    using System.Threading;
+    using System.Threading.Tasks;
+    using Apache.Ignite.Core.Common;
+    using Apache.Ignite.Core.Configuration;
+    using NUnit.Framework;
+
+    /// <summary>
+    /// Tests for <see cref="IIgniteLock"/>.
+    /// </summary>
+    public class IgniteLockTests : TestBase
+    {
+        /// <summary>
+        /// Initializes a new instance of <see cref="IgniteLockTests"/> class.
+        /// </summary>
+        public IgniteLockTests() : base(2)
+        {
+            // No-op.
+        }
+
+        /// <summary>
+        /// Tests state changes: unlocked -> locked -> disposed.
+        /// </summary>
+        [Test]
+        public void TestStateChanges()
+        {
+            var lck = Ignite.GetOrCreateLock(TestUtils.TestName);
+            var cfg = lck.Configuration;
+
+            Assert.IsFalse(cfg.IsFailoverSafe);
+            Assert.IsFalse(cfg.IsFair);
+            Assert.AreEqual(TestUtils.TestName, cfg.Name);
+
+            Assert.False(lck.IsEntered());
+            Assert.False(lck.IsBroken());
+
+            Assert.IsTrue(lck.TryEnter());
+
+            Assert.IsTrue(lck.IsEntered());
+            Assert.False(lck.IsBroken());
+
+            lck.Exit();
+
+            Assert.False(lck.IsEntered());
+            Assert.False(lck.IsBroken());
+
+            lck.Remove();
+
+            Assert.IsTrue(lck.IsRemoved());
+        }
+
+        /// <summary>
+        /// Tests that thread blocks on Enter until lock is released.
+        /// </summary>
+        [Test]
+        public void TestEnterBlocksWhenLockedByAnotherThread()
+        {
+            long state = 0;
+
+            var lock1 = Ignite.GetOrCreateLock(TestUtils.TestName);
+            lock1.Enter();
+
+            // ReSharper disable once AccessToModifiedClosure
+            var task = Task.Factory.StartNew(() =>
+            {
+                var lock2 = Ignite.GetOrCreateLock(TestUtils.TestName);
+                Interlocked.Increment(ref state);
+                lock2.Enter();
+                Interlocked.Increment(ref state);
+                lock2.Exit();
+                Interlocked.Increment(ref state);
+            });
+
+            TestUtils.WaitForTrueCondition(() => Interlocked.Read(ref state) == 1);
+            Assert.AreEqual(1, Interlocked.Read(ref state));
+
+            lock1.Exit();
+            task.Wait();
+            Assert.AreEqual(3, Interlocked.Read(ref state));
+        }
+
+        /// <summary>
+        /// Tests that Exit throws a meaningful exception when lock in not entered.
+        /// </summary>
+        [Test]
+        public void TestExitThrowsCorrectExceptionWhenNotEntered()
+        {
+            var lock1 = Ignite.GetOrCreateLock(TestUtils.TestName);
+
+            var ex = Assert.Throws<SynchronizationLockException>(() => lock1.Exit());
+            var innerEx = ex.InnerException as JavaException;
+
+            Assert.IsNotNull(innerEx);
+            Assert.AreEqual("java.lang.IllegalMonitorStateException", innerEx.JavaClassName);
+        }
+
+        /// <summary>
+        /// Tests that TryEnter succeeds when lock is not taken.
+        /// </summary>
+        [Test]
+        public void TestTryEnterReturnsTrueWhenUnlocked()
+        {
+            var lock1 = Ignite.GetOrCreateLock(TestUtils.TestName);
+
+            Assert.IsTrue(lock1.TryEnter());
+            Assert.IsTrue(lock1.TryEnter(TimeSpan.Zero));
+            Assert.IsTrue(lock1.TryEnter(TimeSpan.FromMilliseconds(50)));
+
+            lock1.Exit();
+        }
+
+        /// <summary>
+        /// Tests that TryEnter fails when lock is taken by another thread.
+        /// </summary>
+        [Test]
+        public void TestTryEnterReturnsFalseWhenLocked()
+        {
+            var lock1 = Ignite.GetOrCreateLock(TestUtils.TestName);
+            var lock2 = Ignite.GetOrCreateLock(TestUtils.TestName);
+
+            lock1.Enter();
+
+            Task.Factory.StartNew(() =>
+            {
+                Assert.IsFalse(lock2.TryEnter());
+                Assert.IsFalse(lock2.TryEnter(TimeSpan.Zero));
+                Assert.IsFalse(lock2.TryEnter(TimeSpan.FromMilliseconds(50)));
+            }).Wait();
+
+            lock1.Exit();
+        }
+
+        /// <summary>
+        /// Tests that lock can be entered multiple times by the same thread.
+        /// </summary>
+        [Test]
+        public void TestReentrancy()
+        {
+            const int count = 10;
+            var lock1 = Ignite.GetOrCreateLock(TestUtils.TestName);
+
+            for (var i = 0; i < count; i++)
+            {
+                lock1.Enter();
+                Assert.IsTrue(lock1.IsEntered());
+            }
+
+            for (var i = 0; i < count; i++)
+            {
+                Assert.IsTrue(lock1.IsEntered());
+                lock1.Exit();
+            }
+
+            Assert.IsFalse(lock1.IsEntered());
+        }
+
+        /// <summary>
+        /// Tests that removed lock throws correct exception.
+        /// </summary>
+        [Test]
+        public void TestRemovedLockThrowsIgniteException()
+        {
+            var lock1 = Ignite.GetOrCreateLock(TestUtils.TestName);
+            var lock2 = Ignite2.GetOrCreateLock(TestUtils.TestName);
+
+            Assert.IsFalse(lock2.IsEntered());
+            lock1.Remove();
+
+            var ex = Assert.Throws<IgniteException>(() => lock2.Enter());
+            Assert.AreEqual("Failed to find reentrant lock with given name: " + lock2.Configuration.Name, ex.Message);
+        }
+
+        /// <summary>
+        /// Tests that removed lock throws correct exception.
+        /// </summary>
+        [Test]
+        [Ignore("IGNITE-13128")]
+        public void TestRemovedBeforeUseLockThrowsIgniteException()
+        {
+            var lock1 = Ignite.GetOrCreateLock(TestUtils.TestName);
+            var lock2 = Ignite2.GetOrCreateLock(TestUtils.TestName);
+
+            lock1.Remove();
+
+            var ex = Assert.Throws<IgniteException>(() => lock2.Enter());
+            Assert.AreEqual("Failed to find reentrant lock with given name: " + lock2.Configuration.Name, ex.Message);
+        }
+
+        /// <summary>
+        /// Tests that entered lock can't be removed.
+        /// </summary>
+        [Test]
+        public void TestEnteredLockThrowsOnRemove()
+        {
+            var cfg = new LockConfiguration
+            {
+                Name = TestUtils.TestName
+            };
+
+            var lck = Ignite.GetOrCreateLock(cfg, true);
+
+            lck.Enter();
+            Assert.IsTrue(lck.IsEntered());
+
+            var ex = Assert.Throws<IgniteException>(() => lck.Remove());
+            StringAssert.StartsWith("Failed to remove reentrant lock with blocked threads", ex.Message);
+
+            lck.Exit();
+            lck.Remove();
+
+            Assert.IsNull(Ignite.GetOrCreateLock(cfg, false));
+        }
+
+        /// <summary>
+        /// Tests configuration propagation.
+        /// </summary>
+        [Test]
+        public void TestLockConfigurationCantBeModifiedAfterLockCreation()
+        {
+            var cfg = new LockConfiguration
+            {
+                Name = TestUtils.TestName,
+                IsFair = true,
+                IsFailoverSafe = true
+            };
+
+            var lck = Ignite.GetOrCreateLock(cfg, true);
+
+            // Change original instance.
+            cfg.Name = "y";
+            cfg.IsFair = false;
+            cfg.IsFailoverSafe = false;
+
+            // Change returned instance.
+            lck.Configuration.Name = "y";
+            lck.Configuration.IsFair = false;
+            lck.Configuration.IsFailoverSafe = false;
+
+            // Verify: actual config has not changed.
+            Assert.AreEqual(TestUtils.TestName, lck.Configuration.Name);
+            Assert.IsTrue(lck.Configuration.IsFair);
+            Assert.IsTrue(lck.Configuration.IsFailoverSafe);
+        }
+
+        /// <summary>
+        /// Tests that null is returned when lock does not exist and create flag is false.
+        /// </summary>
+        [Test]
+        public void TestGetOrCreateLockReturnsNullOnMissingLockWhenCreateFlagIsNotSet()
+        {
+            Assert.IsNull(Ignite.GetOrCreateLock(new LockConfiguration {Name = TestUtils.TestName}, false));
+        }
+
+        /// <summary>
+        /// Tests that fair lock favors granting access to the longest-waiting thread
+        /// </summary>
+        [Test]
+        public void TestFairLockGuaranteesOrder()
+        {
+            const int count = 50;
+
+            var cfg = new LockConfiguration
+            {
+                Name = TestUtils.TestName,
+                IsFair = true,
+                IsFailoverSafe = true
+            };
+
+            var lck = Ignite.GetOrCreateLock(cfg, true);
+            lck.Enter();
+
+            var locks = new ConcurrentQueue<int>();
+            var threads = new Thread[count];
+
+            var evt = new AutoResetEvent(false);
+
+            for (int i = 0; i < count; i++)
+            {
+                var id = i;
+
+                var thread = new Thread(() =>
+                {
+                    evt.Set();
+                    lck.Enter();
+                    locks.Enqueue(id);
+                    lck.Exit();
+                });
+
+                thread.Start();
+
+                evt.WaitOne();
+
+                Thread.Sleep(50);
+
+                threads[i] = thread;
+            }
+
+            lck.Exit();
+
+            foreach (var thread in threads)
+            {
+                thread.Join();
+            }
+
+            Assert.AreEqual(count, locks.Count);
+            CollectionAssert.IsOrdered(locks);
+        }
+    }
+}
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/TestBase.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/TestBase.cs
index cdccd54..d15a2c4 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/TestBase.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/TestBase.cs
@@ -22,8 +22,19 @@ namespace Apache.Ignite.Core.Tests
     /// <summary>
     /// Code configuration test base.
     /// </summary>
-    public class TestBase
+    public abstract class TestBase
     {
+        /** */
+        private readonly int _gridCount;
+
+        /// <summary>
+        /// Initializes a new instance of <see cref="TestBase"/> class.
+        /// </summary>
+        protected TestBase(int gridCount = 1)
+        {
+            _gridCount = gridCount;
+        }
+
         /// <summary>
         /// Sets up the fixture.
         /// </summary>
@@ -31,6 +42,16 @@ namespace Apache.Ignite.Core.Tests
         public void TestFixtureSetUp()
         {
             Ignition.Start(GetConfig());
+
+            for (var i = 1; i < _gridCount; i++)
+            {
+                var cfg = new IgniteConfiguration(GetConfig())
+                {
+                    IgniteInstanceName = i.ToString()
+                };
+
+                Ignition.Start(cfg);
+            }
         }
 
         /// <summary>
@@ -57,5 +78,13 @@ namespace Apache.Ignite.Core.Tests
         {
             get { return Ignition.GetIgnite(); }
         }
+
+        /// <summary>
+        /// Gets the second Ignite instance, if present.
+        /// </summary>
+        protected IIgnite Ignite2
+        {
+            get { return Ignition.GetIgnite("1"); }
+        }
     }
 }
\ No newline at end of file
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj b/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj
index 09e6c8c..32d5d4f 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj
@@ -83,10 +83,12 @@
     <Compile Include="Client\IIgniteClient.cs" />
     <Compile Include="Common\ExceptionFactory.cs" />
     <Compile Include="Configuration\ExecutorConfiguration.cs" />
+    <Compile Include="Configuration\LockConfiguration.cs" />
     <Compile Include="Failure\IFailureHandler.cs" />
     <Compile Include="Failure\NoOpFailureHandler.cs" />
     <Compile Include="Failure\StopNodeFailureHandler.cs" />
     <Compile Include="Failure\StopNodeOrHaltFailureHandler.cs" />
+    <Compile Include="IIgniteLock.cs" />
     <Compile Include="Impl\Binary\BinaryHashCodeUtils.cs" />
     <Compile Include="Impl\Binary\IgniteBiTuple.cs" />
     <Compile Include="Impl\Cache\Platform\IPlatformCache.cs" />
@@ -120,6 +122,7 @@
     <Compile Include="Impl\Common\PlatformType.cs" />
     <Compile Include="Impl\Common\TaskRunner.cs" />
     <Compile Include="Impl\Compute\ComputeRunner.cs" />
+    <Compile Include="Impl\IgniteLock.cs" />
     <Compile Include="Impl\Log\NoopLogger.cs" />
     <Compile Include="Impl\Transactions\TransactionCollectionImpl.cs" />
     <Compile Include="Impl\Unmanaged\UnmanagedThread.cs" />
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Configuration/LockConfiguration.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Configuration/LockConfiguration.cs
new file mode 100644
index 0000000..171b7d6
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Configuration/LockConfiguration.cs
@@ -0,0 +1,68 @@
+/*
+ * 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.Configuration
+{
+    /// <summary>
+    /// <see cref="IIgniteLock"/> configuration.
+    /// </summary>
+    public class LockConfiguration
+    {
+        /// <summary>
+        /// Initializes a new instance of <see cref="LockConfiguration"/> class.
+        /// </summary>
+        public LockConfiguration()
+        {
+            // No-op.
+        }
+
+        /// <summary>
+        /// Initializes a new instance of <see cref="LockConfiguration"/> class.
+        /// </summary>
+        /// <param name="other">Other configuration to copy.</param>
+        public LockConfiguration(LockConfiguration other)
+        {
+            Name = other.Name;
+            IsFair = other.IsFair;
+            IsFailoverSafe = other.IsFailoverSafe;
+        }
+
+        /// <summary>
+        /// Gets or sets the cluster-wide lock name.
+        /// </summary>
+        public string Name { get; set; }
+
+        /// <summary>
+        /// Gets or sets a value indicating whether the lock should be in fair mode.
+        /// <para />
+        /// When true, under contention, locks favor granting access to the longest-waiting thread. Otherwise this
+        /// lock does not guarantee any particular access order.
+        /// <para />
+        /// Fair locks accessed by many threads may display lower overall throughput than those using the default
+        /// setting, but have smaller variances in times to obtain locks and guarantee lack of starvation.
+        /// </summary>
+        public bool IsFair { get; set; }
+
+        /// <summary>
+        /// Gets or sets a value indicating whether the lock should be failover-safe: when true,
+        /// if any node leaves topology, all locks already acquired by that node are silently released
+        /// and become available for other nodes to acquire. When false, all threads on other nodes
+        /// waiting to acquire the lock are interrupted.
+        /// </summary>
+        public bool IsFailoverSafe { get; set; }
+    }
+}
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/IIgnite.cs b/modules/platforms/dotnet/Apache.Ignite.Core/IIgnite.cs
index 21fb1e3..a220160 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/IIgnite.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/IIgnite.cs
@@ -122,7 +122,7 @@ namespace Apache.Ignite.Core
         /// <param name="configuration">Cache configuration.</param>
         /// /// <param name="nearConfiguration">Near cache configuration for client.</param>
         /// <returns>Existing or newly created cache.</returns>
-        ICache<TK, TV> GetOrCreateCache<TK, TV>(CacheConfiguration configuration, 
+        ICache<TK, TV> GetOrCreateCache<TK, TV>(CacheConfiguration configuration,
             NearCacheConfiguration nearConfiguration);
 
         /// <summary>
@@ -136,7 +136,7 @@ namespace Apache.Ignite.Core
         /// When not null, native .NET cache is created additionally.</param>
         /// <returns>Existing or newly created cache.</returns>
         [IgniteExperimental]
-        ICache<TK, TV> GetOrCreateCache<TK, TV>(CacheConfiguration configuration, 
+        ICache<TK, TV> GetOrCreateCache<TK, TV>(CacheConfiguration configuration,
             NearCacheConfiguration nearConfiguration, PlatformCacheConfiguration platformCacheConfiguration);
 
         /// <summary>
@@ -165,7 +165,7 @@ namespace Apache.Ignite.Core
         /// <param name="configuration">Cache configuration.</param>
         /// <param name="nearConfiguration">Near cache configuration for client.</param>
         /// <returns>Existing or newly created cache.</returns>
-        ICache<TK, TV> CreateCache<TK, TV>(CacheConfiguration configuration, 
+        ICache<TK, TV> CreateCache<TK, TV>(CacheConfiguration configuration,
             NearCacheConfiguration nearConfiguration);
 
         /// <summary>
@@ -179,11 +179,11 @@ namespace Apache.Ignite.Core
         /// When not null, native .NET cache is created additionally.</param>
         /// <returns>Existing or newly created cache.</returns>
         [IgniteExperimental]
-        ICache<TK, TV> CreateCache<TK, TV>(CacheConfiguration configuration, 
+        ICache<TK, TV> CreateCache<TK, TV>(CacheConfiguration configuration,
             NearCacheConfiguration nearConfiguration, PlatformCacheConfiguration platformCacheConfiguration);
 
         /// <summary>
-        /// Destroys dynamically created (with <see cref="CreateCache{TK,TV}(string)"/> or 
+        /// Destroys dynamically created (with <see cref="CreateCache{TK,TV}(string)"/> or
         /// <see cref="GetOrCreateCache{TK,TV}(string)"/>) cache.
         /// </summary>
         /// <param name="name">The name of the cache to stop.</param>
@@ -248,7 +248,7 @@ namespace Apache.Ignite.Core
         /// Initial value for the atomic long. Ignored if <c>create</c> is false.
         /// </param>
         /// <param name="create">Flag indicating whether atomic long should be created if it does not exist.</param>
-        /// <returns>Atomic long instance with specified name, 
+        /// <returns>Atomic long instance with specified name,
         /// or null if it does not exist and <c>create</c> flag is not set.</returns>
         /// <exception cref="IgniteException">If atomic long could not be fetched or created.</exception>
         IAtomicLong GetAtomicLong(string name, long initialValue, bool create);
@@ -262,7 +262,7 @@ namespace Apache.Ignite.Core
         /// Initial value for the atomic sequence. Ignored if <paramref name="create"/> is false.
         /// </param>
         /// <param name="create">Flag indicating whether atomic sequence should be created if it does not exist.</param>
-        /// <returns>Atomic sequence instance with specified name, 
+        /// <returns>Atomic sequence instance with specified name,
         /// or null if it does not exist and <paramref name="create"/> flag is not set.</returns>
         /// <exception cref="IgniteException">If atomic sequence could not be fetched or created.</exception>
         IAtomicSequence GetAtomicSequence(string name, long initialValue, bool create);
@@ -276,7 +276,7 @@ namespace Apache.Ignite.Core
         /// Initial value for the atomic reference. Ignored if <paramref name="create"/> is false.
         /// </param>
         /// <param name="create">Flag indicating whether atomic reference should be created if it does not exist.</param>
-        /// <returns>Atomic reference instance with specified name, 
+        /// <returns>Atomic reference instance with specified name,
         /// or null if it does not exist and <paramref name="create"/> flag is not set.</returns>
         /// <exception cref="IgniteException">If atomic reference could not be fetched or created.</exception>
         IAtomicReference<T> GetAtomicReference<T>(string name, T initialValue, bool create);
@@ -442,7 +442,7 @@ namespace Apache.Ignite.Core
         IPersistentStoreMetrics GetPersistentStoreMetrics();
 
         /// <summary>
-        /// Gets a collection of memory metrics, one for each 
+        /// Gets a collection of memory metrics, one for each
         /// <see cref="DataStorageConfiguration.DataRegionConfigurations"/>.
         /// <para />
         /// Metrics should be enabled with <see cref="DataStorageConfiguration.MetricsEnabled"/>.
@@ -473,5 +473,20 @@ namespace Apache.Ignite.Core
         /// </summary>
         /// <param name="configuration">Configuration.</param>
         void AddCacheConfiguration(CacheConfiguration configuration);
+
+        /// <summary>
+        /// Gets or creates a distributed reentrant lock (monitor) with default configuration.
+        /// </summary>
+        /// <param name="name">Lock name.</param>
+        /// <returns><see cref="IIgniteLock"/></returns>
+        IIgniteLock GetOrCreateLock(string name);
+
+        /// <summary>
+        /// Gets or creates a distributed reentrant lock (monitor).
+        /// </summary>
+        /// <param name="configuration">Lock configuration.</param>
+        /// <param name="create">Whether the lock should be created if it does not exist.</param>
+        /// <returns><see cref="IIgniteLock"/></returns>
+        IIgniteLock GetOrCreateLock(LockConfiguration configuration, bool create);
     }
 }
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/IIgniteLock.cs b/modules/platforms/dotnet/Apache.Ignite.Core/IIgniteLock.cs
new file mode 100644
index 0000000..02a6c93
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/IIgniteLock.cs
@@ -0,0 +1,81 @@
+/*
+ * 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
+{
+    using System;
+    using System.Threading;
+    using Apache.Ignite.Core.Configuration;
+
+    /// <summary>
+    /// Distributed reentrant monitor (lock).
+    /// <para />
+    /// The functionality is similar to the standard <see cref="Monitor"/> class, but works across all cluster nodes.
+    /// <para />
+    /// This API corresponds to <c>IgniteLock</c> in Java.
+    /// </summary>
+    public interface IIgniteLock
+    {
+        /// <summary>
+        /// Gets the lock configuration.
+        /// </summary>
+        LockConfiguration Configuration { get; }
+
+        /// <summary>
+        /// Acquires the distributed reentrant lock.
+        /// </summary>
+        void Enter();
+
+        /// <summary>
+        /// Acquires the lock only if it is free at the time of invocation.
+        /// </summary>
+        /// <returns>True if the lock was acquired; false otherwise.</returns>
+        bool TryEnter();
+
+        /// <summary>
+        /// Acquires the lock if it is not held by another thread within the given waiting time.
+        /// </summary>
+        /// <param name="timeout">Time to wait for the lock.</param>
+        /// <returns>True if the lock was acquired; false otherwise.</returns>
+        bool TryEnter(TimeSpan timeout);
+
+        /// <summary>
+        /// Releases the lock.
+        /// </summary>
+        void Exit();
+
+        /// <summary>
+        /// Returns a value indicating whether any node that owned the lock failed before releasing the lock.
+        /// </summary>
+        bool IsBroken();
+
+        /// <summary>
+        /// Determines whether the current thread holds the lock.
+        /// </summary>
+        bool IsEntered();
+
+        /// <summary>
+        /// Removes the lock from the cluster.
+        /// </summary>
+        void Remove();
+
+        /// <summary>
+        /// Gets a value indicating whether the lock has been removed from the cluster.
+        /// </summary>
+        bool IsRemoved();
+    }
+}
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/ExceptionUtils.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/ExceptionUtils.cs
index 6d4aea4..12b6de4 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/ExceptionUtils.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/ExceptionUtils.cs
@@ -66,6 +66,7 @@ namespace Apache.Ignite.Core.Impl
             Exs["java.lang.IllegalStateException"] = (c, m, e, i) => new InvalidOperationException(m, e);
             Exs["java.lang.UnsupportedOperationException"] = (c, m, e, i) => new NotSupportedException(m, e);
             Exs["java.lang.InterruptedException"] = (c, m, e, i) => new ThreadInterruptedException(m, e);
+            Exs["java.lang.IllegalMonitorStateException"] = (c, m, e, i) => new SynchronizationLockException(m, e);
 
             // Generic Ignite exceptions.
             Exs["org.apache.ignite.IgniteException"] = (c, m, e, i) => new IgniteException(m, e);
@@ -156,7 +157,7 @@ namespace Apache.Ignite.Core.Impl
                 if (match.Success && Exs.TryGetValue(match.Groups[1].Value, out innerCtor))
                 {
                     return ctor(clsName, msg,
-                        innerCtor(match.Groups[1].Value, match.Groups[2].Value, innerException, ignite), 
+                        innerCtor(match.Groups[1].Value, match.Groups[2].Value, innerException, ignite),
                         ignite);
                 }
 
@@ -199,7 +200,7 @@ namespace Apache.Ignite.Core.Impl
         /// <param name="reader">Reader.</param>
         /// <returns>CachePartialUpdateException.</returns>
         [SuppressMessage("Microsoft.Design", "CA1031:DoNotCatchGeneralExceptionTypes")]
-        private static Exception ProcessCachePartialUpdateException(IIgniteInternal ignite, string msg, 
+        private static Exception ProcessCachePartialUpdateException(IIgniteInternal ignite, string msg,
             string stackTrace, BinaryReader reader)
         {
             if (reader == null)
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Ignite.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Ignite.cs
index d4aa538..7c23d66 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Ignite.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Ignite.cs
@@ -30,6 +30,7 @@ namespace Apache.Ignite.Core.Impl
     using Apache.Ignite.Core.Cluster;
     using Apache.Ignite.Core.Common;
     using Apache.Ignite.Core.Compute;
+    using Apache.Ignite.Core.Configuration;
     using Apache.Ignite.Core.Datastream;
     using Apache.Ignite.Core.DataStructures;
     using Apache.Ignite.Core.Events;
@@ -99,7 +100,8 @@ namespace Apache.Ignite.Core.Impl
             GetBaselineAutoAdjustTimeout = 34,
             SetBaselineAutoAdjustTimeout = 35,
             GetCacheConfig = 36,
-            GetThreadLocal = 37
+            GetThreadLocal = 37,
+            GetOrCreateLock = 38
         }
 
         /** */
@@ -994,6 +996,39 @@ namespace Apache.Ignite.Core.Impl
                 s => configuration.Write(BinaryUtils.Marshaller.StartMarshal(s)));
         }
 
+        /** <inheritdoc /> */
+        public IIgniteLock GetOrCreateLock(string name)
+        {
+            IgniteArgumentCheck.NotNullOrEmpty(name, "name");
+
+            var configuration = new LockConfiguration
+            {
+                Name = name
+            };
+            
+            return GetOrCreateLock(configuration, true);
+        }
+
+        /** <inheritdoc /> */
+        public IIgniteLock GetOrCreateLock(LockConfiguration configuration, bool create)
+        {
+            IgniteArgumentCheck.NotNull(configuration, "configuration");
+            IgniteArgumentCheck.NotNullOrEmpty(configuration.Name, "configuration.Name");
+            
+            // Create a copy to ignore modifications from outside.
+            var cfg = new LockConfiguration(configuration);
+
+            var target = DoOutOpObject((int) Op.GetOrCreateLock, w =>
+            {
+                w.WriteString(configuration.Name);
+                w.WriteBoolean(configuration.IsFailoverSafe);
+                w.WriteBoolean(configuration.IsFair);
+                w.WriteBoolean(create);
+            });
+            
+            return target == null ? null : new IgniteLock(target, cfg);
+        }
+
         /// <summary>
         /// Gets or creates near cache.
         /// </summary>
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/IgniteLock.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/IgniteLock.cs
new file mode 100644
index 0000000..23ee4df
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/IgniteLock.cs
@@ -0,0 +1,114 @@
+/*
+ * 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
+{
+    using System;
+    using System.Diagnostics;
+    using Apache.Ignite.Core.Configuration;
+
+    /// <summary>
+    /// Ignite distributed reentrant lock.
+    /// </summary>
+    internal class IgniteLock : PlatformTargetAdapter, IIgniteLock
+    {
+        /// <summary>
+        /// Lock operations.
+        /// </summary>
+        private enum Op
+        {
+            Lock = 1,
+            TryLock = 2,
+            Unlock = 3,
+            Remove = 4,
+            IsBroken = 5,
+            IsLocked = 6,
+            IsRemoved = 7
+        }
+
+        /** */
+        private const long TimeoutInfinite = -1;
+
+        /** */
+        private readonly LockConfiguration _cfg;
+
+        /// <summary>
+        /// Initializes a new instance of <see cref="IgniteLock"/>.
+        /// </summary>
+        public IgniteLock(IPlatformTargetInternal target, LockConfiguration cfg)
+            : base(target)
+        {
+            Debug.Assert(cfg != null);
+
+            _cfg = cfg;
+        }
+
+        /** <inheritDoc /> */
+        public LockConfiguration Configuration
+        {
+            get { return new LockConfiguration(_cfg); }
+        }
+
+        /** <inheritDoc /> */
+        public void Enter()
+        {
+            Target.InLongOutLong((int) Op.Lock, 0);
+        }
+
+        /** <inheritDoc /> */
+        public bool TryEnter()
+        {
+            return Target.InLongOutLong((int) Op.TryLock, TimeoutInfinite) == True;
+        }
+
+        /** <inheritDoc /> */
+        public bool TryEnter(TimeSpan timeout)
+        {
+            return Target.InLongOutLong((int) Op.TryLock, (long) timeout.TotalMilliseconds) == True;
+        }
+
+        /** <inheritDoc /> */
+        public void Exit()
+        {
+            Target.InLongOutLong((int) Op.Unlock, 0);
+        }
+
+        /** <inheritDoc /> */
+        public bool IsBroken()
+        {
+            return Target.InLongOutLong((int) Op.IsBroken, 0) == True;
+        }
+
+        /** <inheritDoc /> */
+        public bool IsEntered()
+        {
+            return Target.InLongOutLong((int) Op.IsLocked, 0) == True;
+        }
+
+        /** <inheritDoc /> */
+        public void Remove()
+        {
+            Target.InLongOutLong((int) Op.Remove, 0);
+        }
+
+        /** <inheritDoc /> */
+        public bool IsRemoved()
+        {
+            return Target.InLongOutLong((int) Op.IsRemoved, 0) == True;
+        }
+    }
+}
diff --git a/modules/platforms/dotnet/Apache.Ignite.DotNetCore.sln.DotSettings b/modules/platforms/dotnet/Apache.Ignite.DotNetCore.sln.DotSettings
index c74538c..3bbc7bd 100644
--- a/modules/platforms/dotnet/Apache.Ignite.DotNetCore.sln.DotSettings
+++ b/modules/platforms/dotnet/Apache.Ignite.DotNetCore.sln.DotSettings
@@ -10,4 +10,6 @@
 	<s:Boolean x:Key="/Default/Environment/UnitTesting/ShadowCopy/@EntryValue">False</s:Boolean>
 	<s:Boolean x:Key="/Default/UserDictionary/Words/=failover/@EntryIndexedValue">True</s:Boolean>
 	<s:Boolean x:Key="/Default/UserDictionary/Words/=Multithreaded/@EntryIndexedValue">True</s:Boolean>
+	<s:Boolean x:Key="/Default/UserDictionary/Words/=Reentrancy/@EntryIndexedValue">True</s:Boolean>
+	<s:Boolean x:Key="/Default/UserDictionary/Words/=reentrant/@EntryIndexedValue">True</s:Boolean>
 </wpf:ResourceDictionary>
\ No newline at end of file