Return-Path: X-Original-To: apmail-ignite-commits-archive@minotaur.apache.org Delivered-To: apmail-ignite-commits-archive@minotaur.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id ED67519113 for ; Mon, 11 Apr 2016 14:26:53 +0000 (UTC) Received: (qmail 49211 invoked by uid 500); 11 Apr 2016 14:26:53 -0000 Delivered-To: apmail-ignite-commits-archive@ignite.apache.org Received: (qmail 48926 invoked by uid 500); 11 Apr 2016 14:26:53 -0000 Mailing-List: contact commits-help@ignite.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@ignite.apache.org Delivered-To: mailing list commits@ignite.apache.org Received: (qmail 45244 invoked by uid 99); 11 Apr 2016 14:25:33 -0000 Received: from git1-us-west.apache.org (HELO git1-us-west.apache.org) (140.211.11.23) by apache.org (qpsmtpd/0.29) with ESMTP; Mon, 11 Apr 2016 14:25:33 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id D9462EAB69; Mon, 11 Apr 2016 14:25:29 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 8bit From: vozerov@apache.org To: commits@ignite.apache.org Date: Mon, 11 Apr 2016 14:26:11 -0000 Message-Id: In-Reply-To: <15dbdc0d71374accaf29cbb3b7b59af4@git.apache.org> References: <15dbdc0d71374accaf29cbb3b7b59af4@git.apache.org> X-Mailer: ASF-Git Admin Mailer Subject: [44/50] [abbrv] ignite git commit: IGNITE-2908: .NET: Added TransactionConfiguration, AtomicConfiguration, user attributes and "isDaemon" flag. This closes #595. IGNITE-2908: .NET: Added TransactionConfiguration, AtomicConfiguration, user attributes and "isDaemon" flag. This closes #595. Project: http://git-wip-us.apache.org/repos/asf/ignite/repo Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/3bcabaa5 Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/3bcabaa5 Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/3bcabaa5 Branch: refs/heads/ignite-1786 Commit: 3bcabaa5ec9d8d2041ebf89db2d16569856dc406 Parents: 4367d3d Author: Pavel Tupitsyn Authored: Mon Apr 11 14:30:25 2016 +0300 Committer: vozerov-gridgain Committed: Mon Apr 11 14:30:25 2016 +0300 ---------------------------------------------------------------------- .../utils/PlatformConfigurationUtils.java | 78 ++++++++++++++ .../IgniteConfigurationSerializerTest.cs | 48 +++++++-- .../IgniteConfigurationTest.cs | 36 ++++++- .../Apache.Ignite.Core.csproj | 3 + .../Configuration/AtomicConfiguration.cs | 58 ++++++++++ .../Apache.Ignite.Core/IgniteConfiguration.cs | 108 ++++++++++++++++++- .../IgniteConfigurationSection.xsd | 24 +++++ .../Impl/Cluster/ClusterNodeImpl.cs | 2 +- .../Common/IgniteConfigurationXmlSerializer.cs | 67 +++++++++++- .../Impl/Common/ObjectStringConverter.cs | 104 ++++++++++++++++++ .../Transactions/ITransaction.cs | 3 +- .../Transactions/TransactionConfiguration.cs | 76 +++++++++++++ 12 files changed, 596 insertions(+), 11 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/ignite/blob/3bcabaa5/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformConfigurationUtils.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformConfigurationUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformConfigurationUtils.java index e08d1ba..51c7c6c 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformConfigurationUtils.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformConfigurationUtils.java @@ -28,9 +28,11 @@ import org.apache.ignite.cache.CacheWriteSynchronizationMode; import org.apache.ignite.cache.QueryEntity; import org.apache.ignite.cache.QueryIndex; import org.apache.ignite.cache.QueryIndexType; +import org.apache.ignite.configuration.AtomicConfiguration; import org.apache.ignite.configuration.BinaryConfiguration; import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.configuration.TransactionConfiguration; import org.apache.ignite.internal.binary.*; import org.apache.ignite.platform.dotnet.PlatformDotNetBinaryConfiguration; import org.apache.ignite.platform.dotnet.PlatformDotNetBinaryTypeConfiguration; @@ -41,6 +43,8 @@ import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi; import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder; import org.apache.ignite.spi.discovery.tcp.ipfinder.multicast.TcpDiscoveryMulticastIpFinder; import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder; +import org.apache.ignite.transactions.TransactionConcurrency; +import org.apache.ignite.transactions.TransactionIsolation; import java.lang.management.ManagementFactory; import java.net.InetSocketAddress; @@ -259,6 +263,7 @@ import java.util.Map; cfg.setNetworkTimeout(in.readLong()); cfg.setWorkDirectory(in.readString()); cfg.setLocalHost(in.readString()); + cfg.setDaemon(in.readBoolean()); readCacheConfigurations(in, cfg); readDiscoveryConfiguration(in, cfg); @@ -269,6 +274,39 @@ import java.util.Map; cfg.getBinaryConfiguration().setCompactFooter(in.readBoolean()); } + + int attrCnt = in.readInt(); + + if (attrCnt > 0) { + Map attrs = new HashMap<>(attrCnt); + + for (int i = 0; i < attrCnt; i++) + attrs.put(in.readString(), in.readObject()); + + cfg.setUserAttributes(attrs); + } + + if (in.readBoolean()) { + AtomicConfiguration atomic = new AtomicConfiguration(); + + atomic.setAtomicSequenceReserveSize(in.readInt()); + atomic.setBackups(in.readInt()); + atomic.setCacheMode(CacheMode.fromOrdinal(in.readInt())); + + cfg.setAtomicConfiguration(atomic); + } + + if (in.readBoolean()) { + TransactionConfiguration tx = new TransactionConfiguration(); + + tx.setPessimisticTxLogSize(in.readInt()); + tx.setDefaultTxConcurrency(TransactionConcurrency.fromOrdinal(in.readInt())); + tx.setDefaultTxIsolation(TransactionIsolation.fromOrdinal(in.readInt())); + tx.setDefaultTxTimeout(in.readLong()); + tx.setPessimisticTxLogLinger(in.readInt()); + + cfg.setTransactionConfiguration(tx); + } } /** @@ -542,6 +580,7 @@ import java.util.Map; w.writeLong(cfg.getNetworkTimeout()); w.writeString(cfg.getWorkDirectory()); w.writeString(cfg.getLocalHost()); + w.writeBoolean(cfg.isDaemon()); CacheConfiguration[] cacheCfg = cfg.getCacheConfiguration(); @@ -562,6 +601,45 @@ import java.util.Map; if (bc != null) w.writeBoolean(bc.isCompactFooter()); + Map attrs = cfg.getUserAttributes(); + + if (attrs != null) { + w.writeInt(attrs.size()); + + for (Map.Entry e : attrs.entrySet()) { + w.writeString(e.getKey()); + w.writeObject(e.getValue()); + } + } + else + w.writeInt(0); + + AtomicConfiguration atomic = cfg.getAtomicConfiguration(); + + if (atomic != null) { + w.writeBoolean(true); + + w.writeInt(atomic.getAtomicSequenceReserveSize()); + w.writeInt(atomic.getBackups()); + w.writeInt(atomic.getCacheMode().ordinal()); + } + else + w.writeBoolean(false); + + TransactionConfiguration tx = cfg.getTransactionConfiguration(); + + if (tx != null) { + w.writeBoolean(true); + + w.writeInt(tx.getPessimisticTxLogSize()); + w.writeInt(tx.getDefaultTxConcurrency().ordinal()); + w.writeInt(tx.getDefaultTxIsolation().ordinal()); + w.writeLong(tx.getDefaultTxTimeout()); + w.writeInt(tx.getPessimisticTxLogLinger()); + } + else + w.writeBoolean(false); + w.writeString(cfg.getIgniteHome()); w.writeLong(ManagementFactory.getMemoryMXBean().getHeapMemoryUsage().getInit()); http://git-wip-us.apache.org/repos/asf/ignite/blob/3bcabaa5/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationSerializerTest.cs ---------------------------------------------------------------------- diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationSerializerTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationSerializerTest.cs index 6e9effe..ae48e71 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationSerializerTest.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationSerializerTest.cs @@ -33,12 +33,14 @@ namespace Apache.Ignite.Core.Tests using Apache.Ignite.Core.Cache.Configuration; using Apache.Ignite.Core.Cache.Store; using Apache.Ignite.Core.Common; + using Apache.Ignite.Core.DataStructures.Configuration; using Apache.Ignite.Core.Discovery.Tcp; using Apache.Ignite.Core.Discovery.Tcp.Multicast; using Apache.Ignite.Core.Events; using Apache.Ignite.Core.Impl.Common; using Apache.Ignite.Core.Lifecycle; using Apache.Ignite.Core.Tests.Binary; + using Apache.Ignite.Core.Transactions; using NUnit.Framework; /// @@ -52,7 +54,7 @@ namespace Apache.Ignite.Core.Tests [Test] public void TestPredefinedXml() { - var xml = @" + var xml = @" 127.1.1.1 @@ -94,6 +96,9 @@ namespace Apache.Ignite.Core.Tests TaskFailed JobFinished + + + "; var reader = XmlReader.Create(new StringReader(xml)); @@ -101,6 +106,7 @@ namespace Apache.Ignite.Core.Tests Assert.AreEqual("c:", cfg.WorkDirectory); Assert.AreEqual("127.1.1.1", cfg.Localhost); + Assert.IsTrue(cfg.IsDaemon); Assert.AreEqual(1024, cfg.JvmMaxMemoryMb); Assert.AreEqual(TimeSpan.FromSeconds(10), cfg.MetricsLogFrequency); Assert.AreEqual(TimeSpan.FromMinutes(1), ((TcpDiscoverySpi)cfg.DiscoverySpi).JoinTimeout); @@ -133,6 +139,20 @@ namespace Apache.Ignite.Core.Tests Assert.AreEqual(QueryIndexType.Geospatial, queryEntity.Indexes.Single().IndexType); Assert.AreEqual("indexFld", queryEntity.Indexes.Single().Fields.Single().Name); Assert.AreEqual(true, queryEntity.Indexes.Single().Fields.Single().IsDescending); + + Assert.AreEqual(new Dictionary {{"myNode", "true"}}, cfg.UserAttributes); + + var atomicCfg = cfg.AtomicConfiguration; + Assert.AreEqual(2, atomicCfg.Backups); + Assert.AreEqual(CacheMode.Local, atomicCfg.CacheMode); + Assert.AreEqual(250, atomicCfg.AtomicSequenceReserveSize); + + var tx = cfg.TransactionConfiguration; + Assert.AreEqual(TransactionConcurrency.Optimistic, tx.DefaultTransactionConcurrency); + Assert.AreEqual(TransactionIsolation.RepeatableRead, tx.DefaultTransactionIsolation); + Assert.AreEqual(new TimeSpan(0,1,2), tx.DefaultTimeout); + Assert.AreEqual(15, tx.PessimisticTransactionLogSize); + Assert.AreEqual(TimeSpan.FromSeconds(33), tx.PessimisticTransactionLogLinger); } /// @@ -248,7 +268,7 @@ namespace Apache.Ignite.Core.Tests return; } - var props = type.GetProperties(); + var props = type.GetProperties().Where(p => p.GetIndexParameters().Length == 0); foreach (var propInfo in props) { @@ -307,7 +327,7 @@ namespace Apache.Ignite.Core.Tests Serializer = new TestSerializer() } }, - Types = new [] {typeof(string).FullName }, + Types = new[] {typeof (string).FullName}, DefaultIdMapper = new IdMapper(), DefaultKeepDeserialized = true, DefaultNameMapper = new NameMapper(), @@ -348,7 +368,7 @@ namespace Apache.Ignite.Core.Tests }, Indexes = new[] { - new QueryIndex("field") { IndexType = QueryIndexType.FullText } + new QueryIndex("field") {IndexType = QueryIndexType.FullText} }, Aliases = new[] { @@ -399,7 +419,7 @@ namespace Apache.Ignite.Core.Tests JvmDllPath = @"c:\jvm", JvmInitialMemoryMb = 1024, JvmMaxMemoryMb = 2048, - LifecycleBeans = new[] {new LifecycleBean(), new LifecycleBean() }, + LifecycleBeans = new[] {new LifecycleBean(), new LifecycleBean()}, MetricsExpireTime = TimeSpan.FromSeconds(15), MetricsHistorySize = 45, MetricsLogFrequency = TimeSpan.FromDays(2), @@ -408,7 +428,23 @@ namespace Apache.Ignite.Core.Tests NetworkSendRetryDelay = TimeSpan.FromSeconds(98), NetworkTimeout = TimeSpan.FromMinutes(4), SuppressWarnings = true, - WorkDirectory = @"c:\work" + WorkDirectory = @"c:\work", + IsDaemon = true, + UserAttributes = Enumerable.Range(1, 10).ToDictionary(x => x.ToString(), x => (object) x), + AtomicConfiguration = new AtomicConfiguration + { + CacheMode = CacheMode.Replicated, + AtomicSequenceReserveSize = 200, + Backups = 2 + }, + TransactionConfiguration = new TransactionConfiguration + { + PessimisticTransactionLogSize = 23, + DefaultTransactionIsolation = TransactionIsolation.ReadCommitted, + DefaultTimeout = TimeSpan.FromDays(2), + DefaultTransactionConcurrency = TransactionConcurrency.Optimistic, + PessimisticTransactionLogLinger = TimeSpan.FromHours(3) + } }; } http://git-wip-us.apache.org/repos/asf/ignite/blob/3bcabaa5/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationTest.cs ---------------------------------------------------------------------- diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationTest.cs index c1f8fcd..c6a1cab 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationTest.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/IgniteConfigurationTest.cs @@ -25,10 +25,12 @@ namespace Apache.Ignite.Core.Tests using Apache.Ignite.Core.Binary; using Apache.Ignite.Core.Cache.Configuration; using Apache.Ignite.Core.Common; + using Apache.Ignite.Core.DataStructures.Configuration; using Apache.Ignite.Core.Discovery.Tcp; using Apache.Ignite.Core.Discovery.Tcp.Multicast; using Apache.Ignite.Core.Discovery.Tcp.Static; using Apache.Ignite.Core.Events; + using Apache.Ignite.Core.Transactions; using NUnit.Framework; /// @@ -108,6 +110,22 @@ namespace Apache.Ignite.Core.Tests Assert.AreEqual(cfg.JvmOptions, resCfg.JvmOptions); Assert.IsTrue(File.Exists(resCfg.JvmDllPath)); Assert.AreEqual(cfg.Localhost, resCfg.Localhost); + Assert.AreEqual(cfg.IsDaemon, resCfg.IsDaemon); + Assert.AreEqual(cfg.UserAttributes, resCfg.UserAttributes); + + var atm = cfg.AtomicConfiguration; + var resAtm = resCfg.AtomicConfiguration; + Assert.AreEqual(atm.AtomicSequenceReserveSize, resAtm.AtomicSequenceReserveSize); + Assert.AreEqual(atm.Backups, resAtm.Backups); + Assert.AreEqual(atm.CacheMode, resAtm.CacheMode); + + var tx = cfg.TransactionConfiguration; + var resTx = resCfg.TransactionConfiguration; + Assert.AreEqual(tx.DefaultTimeout, resTx.DefaultTimeout); + Assert.AreEqual(tx.DefaultTransactionConcurrency, resTx.DefaultTransactionConcurrency); + Assert.AreEqual(tx.DefaultTransactionIsolation, resTx.DefaultTransactionIsolation); + Assert.AreEqual(tx.PessimisticTransactionLogLinger, resTx.PessimisticTransactionLogLinger); + Assert.AreEqual(tx.PessimisticTransactionLogSize, resTx.PessimisticTransactionLogSize); } } @@ -349,7 +367,23 @@ namespace Apache.Ignite.Core.Tests WorkDirectory = Path.GetTempPath(), JvmOptions = TestUtils.TestJavaOptions(), JvmClasspath = TestUtils.CreateTestClasspath(), - Localhost = "127.0.0.1" + Localhost = "127.0.0.1", + IsDaemon = true, + UserAttributes = Enumerable.Range(1, 10).ToDictionary(x => x.ToString(), x => (object) x), + AtomicConfiguration = new AtomicConfiguration + { + CacheMode = CacheMode.Replicated, + Backups = 2, + AtomicSequenceReserveSize = 200 + }, + TransactionConfiguration = new TransactionConfiguration + { + DefaultTransactionConcurrency = TransactionConcurrency.Optimistic, + DefaultTimeout = TimeSpan.FromSeconds(25), + DefaultTransactionIsolation = TransactionIsolation.Serializable, + PessimisticTransactionLogLinger = TimeSpan.FromHours(1), + PessimisticTransactionLogSize = 240 + } }; } } http://git-wip-us.apache.org/repos/asf/ignite/blob/3bcabaa5/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj ---------------------------------------------------------------------- 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 9a5211c..a2575cc 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj @@ -92,6 +92,7 @@ + @@ -251,6 +252,7 @@ + @@ -431,6 +433,7 @@ + http://git-wip-us.apache.org/repos/asf/ignite/blob/3bcabaa5/modules/platforms/dotnet/Apache.Ignite.Core/DataStructures/Configuration/AtomicConfiguration.cs ---------------------------------------------------------------------- diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/DataStructures/Configuration/AtomicConfiguration.cs b/modules/platforms/dotnet/Apache.Ignite.Core/DataStructures/Configuration/AtomicConfiguration.cs new file mode 100644 index 0000000..c6fcbbf --- /dev/null +++ b/modules/platforms/dotnet/Apache.Ignite.Core/DataStructures/Configuration/AtomicConfiguration.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.DataStructures.Configuration +{ + using System.ComponentModel; + using Apache.Ignite.Core.Cache.Configuration; + + /// + /// Configuration for atomic data structures. + /// + public class AtomicConfiguration + { + /// Default number of backups. + public const int DefaultBackups = 0; + + /// Default caching mode. + public const CacheMode DefaultCacheMode = CacheMode.Partitioned; + + /// Default atomic sequence reservation size. + public const int DefaultAtomicSequenceReserveSize = 1000; + + /// + /// Gets or sets number of nodes used to back up single partition for + /// cache. + /// + [DefaultValue(DefaultBackups)] + public int Backups { get; set; } + + /// + /// Gets or sets caching mode to use. + /// + [DefaultValue(DefaultCacheMode)] + public CacheMode CacheMode { get; set; } + + /// + /// Gets or sets the default number of sequence values reserved for instances. + /// After a certain number has been reserved, consequent increments of sequence will happen locally, + /// without communication with other nodes, until the next reservation has to be made. + /// + [DefaultValue(DefaultAtomicSequenceReserveSize)] + public int AtomicSequenceReserveSize { get; set; } + } +} http://git-wip-us.apache.org/repos/asf/ignite/blob/3bcabaa5/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfiguration.cs ---------------------------------------------------------------------- diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfiguration.cs b/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfiguration.cs index 2aa6899..56d8867 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfiguration.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfiguration.cs @@ -27,6 +27,8 @@ using System.Linq; using Apache.Ignite.Core.Binary; using Apache.Ignite.Core.Cache.Configuration; + using Apache.Ignite.Core.Cluster; + using Apache.Ignite.Core.DataStructures.Configuration; using Apache.Ignite.Core.Discovery; using Apache.Ignite.Core.Discovery.Tcp; using Apache.Ignite.Core.Events; @@ -34,6 +36,7 @@ using Apache.Ignite.Core.Impl.Binary; using Apache.Ignite.Core.Impl.Common; using Apache.Ignite.Core.Lifecycle; + using Apache.Ignite.Core.Transactions; using BinaryReader = Apache.Ignite.Core.Impl.Binary.BinaryReader; using BinaryWriter = Apache.Ignite.Core.Impl.Binary.BinaryWriter; @@ -177,6 +180,7 @@ writer.WriteLong((long) NetworkTimeout.TotalMilliseconds); writer.WriteString(WorkDirectory); writer.WriteString(Localhost); + writer.WriteBoolean(IsDaemon); // Cache config var caches = CacheConfiguration; @@ -215,6 +219,48 @@ if (isCompactFooterSet) writer.WriteBoolean(BinaryConfiguration.CompactFooter); + + // User attributes + var attrs = UserAttributes; + + if (attrs == null) + writer.WriteInt(0); + else + { + writer.WriteInt(attrs.Count); + + foreach (var pair in attrs) + { + writer.WriteString(pair.Key); + writer.Write(pair.Value); + } + } + + // Atomic + if (AtomicConfiguration != null) + { + writer.WriteBoolean(true); + + writer.WriteInt(AtomicConfiguration.AtomicSequenceReserveSize); + writer.WriteInt(AtomicConfiguration.Backups); + writer.WriteInt((int) AtomicConfiguration.CacheMode); + } + else + writer.WriteBoolean(false); + + // Tx + if (TransactionConfiguration != null) + { + writer.WriteBoolean(true); + + writer.WriteInt(TransactionConfiguration.PessimisticTransactionLogSize); + writer.WriteInt((int) TransactionConfiguration.DefaultTransactionConcurrency); + writer.WriteInt((int) TransactionConfiguration.DefaultTransactionIsolation); + writer.WriteLong((long) TransactionConfiguration.DefaultTimeout.TotalMilliseconds); + writer.WriteLong((int) TransactionConfiguration.PessimisticTransactionLogLinger.TotalMilliseconds); + } + else + writer.WriteBoolean(false); } /// @@ -236,6 +282,7 @@ NetworkTimeout = r.ReadLongAsTimespan(); WorkDirectory = r.ReadString(); Localhost = r.ReadString(); + IsDaemon = r.ReadBoolean(); // Cache config var cacheCfgCount = r.ReadInt(); @@ -252,6 +299,34 @@ BinaryConfiguration = BinaryConfiguration ?? new BinaryConfiguration(); BinaryConfiguration.CompactFooter = r.ReadBoolean(); } + + // User attributes + UserAttributes = Enumerable.Range(0, r.ReadInt()) + .ToDictionary(x => r.ReadString(), x => r.ReadObject()); + + // Atomic + if (r.ReadBoolean()) + { + AtomicConfiguration = new AtomicConfiguration + { + AtomicSequenceReserveSize = r.ReadInt(), + Backups = r.ReadInt(), + CacheMode = (CacheMode) r.ReadInt() + }; + } + + // Tx + if (r.ReadBoolean()) + { + TransactionConfiguration = new TransactionConfiguration + { + PessimisticTransactionLogSize = r.ReadInt(), + DefaultTransactionConcurrency = (TransactionConcurrency) r.ReadInt(), + DefaultTransactionIsolation = (TransactionIsolation) r.ReadInt(), + DefaultTimeout = TimeSpan.FromMilliseconds(r.ReadLong()), + PessimisticTransactionLogLinger = TimeSpan.FromMilliseconds(r.ReadInt()) + }; + } } /// @@ -394,7 +469,6 @@ /// /// Gets or sets a value indicating whether node should start in client mode. /// Client node cannot hold data in the caches. - /// Default is null and takes this setting from Spring configuration. /// public bool ClientMode { get; set; } @@ -465,5 +539,37 @@ /// It is strongly recommended to set this parameter for all production environments. /// public string Localhost { get; set; } + + /// + /// Gets or sets a value indicating whether this node should be a daemon node. + /// + /// Daemon nodes are the usual grid nodes that participate in topology but not visible on the main APIs, + /// i.e. they are not part of any cluster groups. + /// + /// Daemon nodes are used primarily for management and monitoring functionality that is built on Ignite + /// and needs to participate in the topology, but also needs to be excluded from the "normal" topology, + /// so that it won't participate in the task execution or in-memory data grid storage. + /// + public bool IsDaemon { get; set; } + + /// + /// Gets or sets the user attributes for this node. + /// + /// These attributes can be retrieved later via . + /// Environment variables are added to node attributes automatically. + /// NOTE: attribute names starting with "org.apache.ignite" are reserved for internal use. + /// + [SuppressMessage("Microsoft.Usage", "CA2227:CollectionPropertiesShouldBeReadOnly")] + public IDictionary UserAttributes { get; set; } + + /// + /// Gets or sets the atomic data structures configuration. + /// + public AtomicConfiguration AtomicConfiguration { get; set; } + + /// + /// Gets or sets the transaction configuration. + /// + public TransactionConfiguration TransactionConfiguration { get; set; } } } http://git-wip-us.apache.org/repos/asf/ignite/blob/3bcabaa5/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfigurationSection.xsd ---------------------------------------------------------------------- diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfigurationSection.xsd b/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfigurationSection.xsd index c7b6cf2..a0df870 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfigurationSection.xsd +++ b/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfigurationSection.xsd @@ -269,6 +269,29 @@ + + + + + + + + + + + + + + + + + + + + + + + @@ -287,6 +310,7 @@ + http://git-wip-us.apache.org/repos/asf/ignite/blob/3bcabaa5/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cluster/ClusterNodeImpl.cs ---------------------------------------------------------------------- diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cluster/ClusterNodeImpl.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cluster/ClusterNodeImpl.cs index 9b83dd5..b658353 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cluster/ClusterNodeImpl.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cluster/ClusterNodeImpl.cs @@ -119,7 +119,7 @@ namespace Apache.Ignite.Core.Impl.Cluster /** */ public IDictionary GetAttributes() { - return _attrs; + return _attrs.AsReadOnly(); } /** */ http://git-wip-us.apache.org/repos/asf/ignite/blob/3bcabaa5/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Common/IgniteConfigurationXmlSerializer.cs ---------------------------------------------------------------------- diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Common/IgniteConfigurationXmlSerializer.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Common/IgniteConfigurationXmlSerializer.cs index c27012a..52fbc30 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Common/IgniteConfigurationXmlSerializer.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Common/IgniteConfigurationXmlSerializer.cs @@ -39,6 +39,9 @@ namespace Apache.Ignite.Core.Impl.Common /** Xmlns. */ private const string XmlnsAttribute = "xmlns"; + /** Xmlns. */ + private const string KeyValPairElement = "pair"; + /** Schema. */ private const string Schema = "http://ignite.apache.org/schema/dotnet/IgniteConfigurationSection"; @@ -89,6 +92,8 @@ namespace Apache.Ignite.Core.Impl.Common WriteBasicProperty(obj, writer, valueType, property); else if (valueType.IsGenericType && valueType.GetGenericTypeDefinition() == typeof (ICollection<>)) WriteCollectionProperty(obj, writer, valueType, property); + else if (valueType.IsGenericType && valueType.GetGenericTypeDefinition() == typeof (IDictionary<,>)) + WriteDictionaryProperty(obj, writer, valueType, property); else WriteComplexProperty(obj, writer, valueType); @@ -121,6 +126,17 @@ namespace Apache.Ignite.Core.Impl.Common } /// + /// Writes the dictionary property. + /// + private static void WriteDictionaryProperty(object obj, XmlWriter writer, Type valueType, PropertyInfo property) + { + var elementType = typeof (KeyValuePair<,>).MakeGenericType(valueType.GetGenericArguments()); + + foreach (var element in (IEnumerable)obj) + WriteElement(element, writer, KeyValPairElement, elementType, property); + } + + /// /// Writes the complex property (nested object). /// private static void WriteComplexProperty(object obj, XmlWriter writer, Type valueType) @@ -183,6 +199,11 @@ namespace Apache.Ignite.Core.Impl.Common // Collection ReadCollectionProperty(reader, prop, target); } + else if (propType.IsGenericType && propType.GetGenericTypeDefinition() == typeof (IDictionary<,>)) + { + // Dictionary + ReadDictionaryProperty(reader, prop, target); + } else { // Nested object (complex property) @@ -268,6 +289,43 @@ namespace Apache.Ignite.Core.Impl.Common prop.SetValue(target, list, null); } + + /// + /// Reads the dictionary. + /// + private static void ReadDictionaryProperty(XmlReader reader, PropertyInfo prop, object target) + { + var keyValTypes = prop.PropertyType.GetGenericArguments(); + + var dictType = typeof (Dictionary<,>).MakeGenericType(keyValTypes); + + var dict = (IDictionary) Activator.CreateInstance(dictType); + + using (var subReader = reader.ReadSubtree()) + { + subReader.Read(); // skip list head + while (subReader.Read()) + { + if (subReader.NodeType != XmlNodeType.Element) + continue; + + if (subReader.Name != PropertyNameToXmlName(KeyValPairElement)) + throw new ConfigurationErrorsException( + string.Format("Invalid dictionary element in IgniteConfiguration: expected '{0}', " + + "but was '{1}'", KeyValPairElement, subReader.Name)); + + var key = subReader.GetAttribute("key"); + + if (key == null) + throw new ConfigurationErrorsException( + "Invalid dictionary entry, key attribute is missing for property " + prop); + + dict[key] = subReader.GetAttribute("value"); + } + } + + prop.SetValue(target, dict, null); + } /// /// Sets the property. @@ -344,7 +402,11 @@ namespace Apache.Ignite.Core.Impl.Common { Debug.Assert(propertyType != null); - return propertyType.IsValueType || propertyType == typeof(string) || propertyType == typeof(Type); + if (propertyType.IsGenericType && propertyType.GetGenericTypeDefinition() == typeof (KeyValuePair<,>)) + return false; + + return propertyType.IsValueType || propertyType == typeof (string) || propertyType == typeof (Type) || + propertyType == typeof (object); } /// @@ -367,6 +429,9 @@ namespace Apache.Ignite.Core.Impl.Common if (property.DeclaringType == typeof (IgniteConfiguration) && property.Name == "IncludedEventTypes") return EventTypeConverter.Instance; + if (propertyType == typeof (object)) + return ObjectStringConverter.Instance; + var converter = TypeDescriptor.GetConverter(propertyType); if (converter == null || !converter.CanConvertFrom(typeof(string)) || http://git-wip-us.apache.org/repos/asf/ignite/blob/3bcabaa5/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Common/ObjectStringConverter.cs ---------------------------------------------------------------------- diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Common/ObjectStringConverter.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Common/ObjectStringConverter.cs new file mode 100644 index 0000000..6961729 --- /dev/null +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Common/ObjectStringConverter.cs @@ -0,0 +1,104 @@ +/* + * 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.Common +{ + using System; + using System.ComponentModel; + using System.Diagnostics.CodeAnalysis; + using System.Globalization; + + /// + /// Object-string converter. + /// + internal class ObjectStringConverter : TypeConverter + { + /// + /// Default instance. + /// + public static readonly ObjectStringConverter Instance = new ObjectStringConverter(); + + /// + /// Returns whether this converter can convert an object of the given type to the type of this converter, + /// using the specified context. + /// + /// An that provides a format context. + /// A that represents the type you want to convert from. + /// + /// true if this converter can perform the conversion; otherwise, false. + /// + public override bool CanConvertFrom(ITypeDescriptorContext context, Type sourceType) + { + return sourceType == typeof(string); + } + + /// + /// Returns whether this converter can convert the object to the specified type, using the specified context. + /// + /// An that provides a format context. + /// + /// A that represents the type you want to convert to. + /// + /// + /// true if this converter can perform the conversion; otherwise, false. + /// + public override bool CanConvertTo(ITypeDescriptorContext context, Type destinationType) + { + return destinationType == typeof(string); + } + + /// + /// Converts the given object to the type of this converter, + /// using the specified context and culture information. + /// + /// An that provides a format context. + /// The to use as the current culture. + /// The to convert. + /// + /// An that represents the converted value. + /// + public override object ConvertFrom(ITypeDescriptorContext context, CultureInfo culture, object value) + { + return value == null ? null : value.ToString(); + } + + /// + /// Converts the given value object to the specified type, using the specified context and culture information. + /// + /// An that provides a format context. + /// + /// A . If null is passed, the current culture is assumed. + /// + /// The to convert. + /// + /// The to convert the parameter to. + /// + /// + /// An that represents the converted value. + /// + [SuppressMessage("Microsoft.Globalization", "CA1308:NormalizeStringsToUppercase")] + [SuppressMessage("Microsoft.Design", "CA1062:Validate arguments of public methods", MessageId = "2")] + public override object ConvertTo(ITypeDescriptorContext context, CultureInfo culture, object value, + Type destinationType) + { + if (destinationType == typeof (string)) + return value.ToString(); + + return base.ConvertTo(context, culture, value, destinationType); + } + } +} http://git-wip-us.apache.org/repos/asf/ignite/blob/3bcabaa5/modules/platforms/dotnet/Apache.Ignite.Core/Transactions/ITransaction.cs ---------------------------------------------------------------------- diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Transactions/ITransaction.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Transactions/ITransaction.cs index 2c062e7..488b85f 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Transactions/ITransaction.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Transactions/ITransaction.cs @@ -166,8 +166,9 @@ namespace Apache.Ignite.Core.Transactions } /// - /// Timeout value in milliseconds for this transaction. If transaction times + /// Timeout for this transaction. If transaction times /// out prior to it's completion, an exception will be thrown. + /// for infinite timeout. /// TimeSpan Timeout { http://git-wip-us.apache.org/repos/asf/ignite/blob/3bcabaa5/modules/platforms/dotnet/Apache.Ignite.Core/Transactions/TransactionConfiguration.cs ---------------------------------------------------------------------- diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Transactions/TransactionConfiguration.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Transactions/TransactionConfiguration.cs new file mode 100644 index 0000000..8703a79 --- /dev/null +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Transactions/TransactionConfiguration.cs @@ -0,0 +1,76 @@ +/* + * 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.Transactions +{ + using System; + using System.ComponentModel; + + /// + /// Transactions configuration. + /// + public class TransactionConfiguration + { + /// The default value for property. + public const TransactionConcurrency DefaultDefaultTransactionConcurrency = TransactionConcurrency.Pessimistic; + + /// The default value for property. + public const TransactionIsolation DefaultDefaultTransactionIsolation = TransactionIsolation.RepeatableRead; + + /// The default value for property. + public static readonly TimeSpan DefaultDefaultTimeout = TimeSpan.Zero; + + /// The default value for property. + public const int DefaultPessimisticTransactionLogSize = 0; + + /// The default value for property. + public static readonly TimeSpan DefaultPessimisticTransactionLogLinger = TimeSpan.FromMilliseconds(10000); + + /// + /// Gets or sets the cache transaction concurrency to use when one is not explicitly specified. + /// + [DefaultValue(DefaultDefaultTransactionConcurrency)] + public TransactionConcurrency DefaultTransactionConcurrency { get; set; } + + /// + /// Gets or sets the cache transaction isolation to use when one is not explicitly specified. + /// + [DefaultValue(DefaultDefaultTransactionIsolation)] + public TransactionIsolation DefaultTransactionIsolation { get; set; } + + /// + /// Gets or sets the cache transaction timeout to use when one is not explicitly specified. + /// for infinite timeout. + /// + [DefaultValue(typeof(TimeSpan), "00:00:00")] + public TimeSpan DefaultTimeout { get; set; } + + /// + /// Gets or sets the size of pessimistic transactions log stored on node in order to recover + /// transaction commit if originating node has left grid before it has sent all messages to transaction nodes. + /// 0 for unlimited. + /// + [DefaultValue(DefaultPessimisticTransactionLogSize)] + public int PessimisticTransactionLogSize { get; set; } + + /// + /// Gets or sets the delay after which pessimistic recovery entries will be cleaned up for failed node. + /// + [DefaultValue(typeof(TimeSpan), "00:00:10")] + public TimeSpan PessimisticTransactionLogLinger { get; set; } + } +}