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 9A49B18C5F for ; Thu, 4 Feb 2016 14:52:07 +0000 (UTC) Received: (qmail 61389 invoked by uid 500); 4 Feb 2016 14:45:11 -0000 Delivered-To: apmail-ignite-commits-archive@ignite.apache.org Received: (qmail 61156 invoked by uid 500); 4 Feb 2016 14:45:11 -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 60368 invoked by uid 99); 4 Feb 2016 14:45:10 -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; Thu, 04 Feb 2016 14:45:10 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id 90337E38C8; Thu, 4 Feb 2016 14:45:10 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 8bit From: sboikov@apache.org To: commits@ignite.apache.org Date: Thu, 04 Feb 2016 14:45:27 -0000 Message-Id: <680b3433aa9f4c23b1e3bd052da86d0e@git.apache.org> In-Reply-To: References: X-Mailer: ASF-Git Admin Mailer Subject: [18/50] [abbrv] ignite git commit: IGNITE-1906: .NET: Implemented programmatic configuration. http://git-wip-us.apache.org/repos/asf/ignite/blob/ee20f1d9/modules/platforms/dotnet/Apache.Ignite.Core/Discovery/Tcp/Multicast/TcpDiscoveryMulticastIpFinder.cs ---------------------------------------------------------------------- diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Discovery/Tcp/Multicast/TcpDiscoveryMulticastIpFinder.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Discovery/Tcp/Multicast/TcpDiscoveryMulticastIpFinder.cs new file mode 100644 index 0000000..25adf56 --- /dev/null +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Discovery/Tcp/Multicast/TcpDiscoveryMulticastIpFinder.cs @@ -0,0 +1,133 @@ +/* + * 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.Discovery.Tcp.Multicast +{ + using System; + using System.ComponentModel; + using Apache.Ignite.Core.Binary; + using Apache.Ignite.Core.Discovery.Tcp.Static; + + /// + /// Multicast-based IP finder. + /// + /// When TCP discovery starts this finder sends multicast request and waits + /// for some time when others nodes reply to this request with messages containing their addresses + /// + public class TcpDiscoveryMulticastIpFinder : TcpDiscoveryStaticIpFinder + { + /// + /// Default multicast port. + /// + public const int DefaultMulticastPort = 47400; + + /// + /// Default address request attempts. + /// + public const int DefaultAddressRequestAttempts = 2; + + /// + /// Default response timeout. + /// + public static readonly TimeSpan DefaultResponseTimeout = TimeSpan.FromMilliseconds(500); + + /// + /// Initializes a new instance of the class. + /// + public TcpDiscoveryMulticastIpFinder() + { + MulticastPort = DefaultMulticastPort; + AddressRequestAttempts = DefaultAddressRequestAttempts; + ResponseTimeout = DefaultResponseTimeout; + } + + /// + /// Gets or sets the local address. + /// If provided address is non-loopback then multicast socket is bound to this interface. + /// If local address is not set or is any local address then IP finder + /// creates multicast sockets for all found non-loopback addresses. + /// + public string LocalAddress { get; set; } + + /// + /// Gets or sets the IP address of the multicast group. + /// + public string MulticastGroup { get; set; } + + /// + /// Gets or sets the port number which multicast messages are sent to. + /// + [DefaultValue(DefaultMulticastPort)] + public int MulticastPort { get; set; } + + /// + /// Gets or sets the number of attempts to send multicast address request. IP finder re-sends + /// request only in case if no reply for previous request is received. + /// + [DefaultValue(DefaultAddressRequestAttempts)] + public int AddressRequestAttempts { get; set; } + + /// + /// Gets or sets the response timeout. + /// + [DefaultValue(typeof(TimeSpan), "00:00:00.5")] + public TimeSpan ResponseTimeout { get; set; } + + /// + /// Gets or sets the time to live for multicast packets sent out on this + /// IP finder in order to control the scope of the multicast. + /// + public byte? TimeToLive { get; set; } // TODO: Nullable? + + /// + /// Initializes a new instance of the class. + /// + /// The reader. + internal TcpDiscoveryMulticastIpFinder(IBinaryRawReader reader) : base(reader) + { + LocalAddress = reader.ReadString(); + MulticastGroup = reader.ReadString(); + MulticastPort = reader.ReadInt(); + AddressRequestAttempts = reader.ReadInt(); + ResponseTimeout = TimeSpan.FromMilliseconds(reader.ReadInt()); + TimeToLive = reader.ReadBoolean() ? (byte?) reader.ReadInt() : null; + } + + /** */ + internal override void Write(IBinaryRawWriter writer) + { + base.Write(writer); + + writer.WriteString(LocalAddress); + writer.WriteString(MulticastGroup); + writer.WriteInt(MulticastPort); + writer.WriteInt(AddressRequestAttempts); + writer.WriteInt((int) ResponseTimeout.TotalMilliseconds); + + writer.WriteBoolean(TimeToLive.HasValue); + + if (TimeToLive.HasValue) + writer.WriteInt(TimeToLive.Value); + } + + /** */ + protected override byte TypeCode + { + get { return TypeCodeMulticastIpFinder; } + } + } +} \ No newline at end of file http://git-wip-us.apache.org/repos/asf/ignite/blob/ee20f1d9/modules/platforms/dotnet/Apache.Ignite.Core/Discovery/Tcp/Static/TcpDiscoveryStaticIpFinder.cs ---------------------------------------------------------------------- diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Discovery/Tcp/Static/TcpDiscoveryStaticIpFinder.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Discovery/Tcp/Static/TcpDiscoveryStaticIpFinder.cs new file mode 100644 index 0000000..331ca48 --- /dev/null +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Discovery/Tcp/Static/TcpDiscoveryStaticIpFinder.cs @@ -0,0 +1,84 @@ +/* + * 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.Discovery.Tcp.Static +{ + using System.Collections.Generic; + using System.Diagnostics.CodeAnalysis; + using Apache.Ignite.Core.Binary; + + /// + /// IP Finder which works only with pre-configured list of IP addresses. + /// + public class TcpDiscoveryStaticIpFinder : TcpDiscoveryIpFinderBase + { + /// + /// Gets or sets the end points. + /// + [SuppressMessage("Microsoft.Usage", "CA2227:CollectionPropertiesShouldBeReadOnly")] + public ICollection Endpoints { get; set; } + + /// + /// Initializes a new instance of the class. + /// + public TcpDiscoveryStaticIpFinder() + { + // No-op. + } + + /// + /// Initializes a new instance of the class. + /// + /// The reader. + internal TcpDiscoveryStaticIpFinder(IBinaryRawReader reader) + { + var count = reader.ReadInt(); + + if (count > 0) + { + Endpoints = new List(count); + + for (int i = 0; i < count; i++) + Endpoints.Add(reader.ReadString()); + } + } + + /** */ + internal override void Write(IBinaryRawWriter writer) + { + base.Write(writer); + + var eps = Endpoints; + + if (eps != null) + { + writer.WriteInt(eps.Count); + + foreach (var ep in eps) + writer.WriteString(ep); + } + else + writer.WriteInt(0); + } + + /** */ + protected override byte TypeCode + { + get { return TypeCodeVmIpFinder; } + } + } +} \ No newline at end of file http://git-wip-us.apache.org/repos/asf/ignite/blob/ee20f1d9/modules/platforms/dotnet/Apache.Ignite.Core/Discovery/Tcp/TcpDiscoveryIpFinderBase.cs ---------------------------------------------------------------------- diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Discovery/Tcp/TcpDiscoveryIpFinderBase.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Discovery/Tcp/TcpDiscoveryIpFinderBase.cs new file mode 100644 index 0000000..e06a88d --- /dev/null +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Discovery/Tcp/TcpDiscoveryIpFinderBase.cs @@ -0,0 +1,78 @@ +/* + * 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.Discovery.Tcp +{ + using Apache.Ignite.Core.Binary; + using Apache.Ignite.Core.Discovery.Tcp.Multicast; + using Apache.Ignite.Core.Discovery.Tcp.Static; + + /// + /// Base IpFinder class. + /// + public abstract class TcpDiscoveryIpFinderBase : ITcpDiscoveryIpFinder + { + /** */ + protected const byte TypeCodeVmIpFinder = 1; + + /** */ + protected const byte TypeCodeMulticastIpFinder = 2; + + /// + /// Initializes a new instance of the class. + /// Prevents user-defined implementations. + /// + protected internal TcpDiscoveryIpFinderBase() + { + // No-op. + } + + /// + /// Writes this instance to the specified writer. + /// + /// The writer. + internal virtual void Write(IBinaryRawWriter writer) + { + writer.WriteByte(TypeCode); + } + + /// + /// Gets the type code to be used in Java to determine ip finder type. + /// + protected abstract byte TypeCode { get; } + + /// + /// Reads the instance. + /// + internal static TcpDiscoveryIpFinderBase ReadInstance(IBinaryRawReader reader) + { + var code = reader.ReadByte(); + + switch (code) + { + case TypeCodeVmIpFinder: + return new TcpDiscoveryStaticIpFinder(reader); + + case TypeCodeMulticastIpFinder: + return new TcpDiscoveryMulticastIpFinder(reader); + + default: + return null; + } + } + } +} \ No newline at end of file http://git-wip-us.apache.org/repos/asf/ignite/blob/ee20f1d9/modules/platforms/dotnet/Apache.Ignite.Core/Discovery/Tcp/TcpDiscoverySpi.cs ---------------------------------------------------------------------- diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Discovery/Tcp/TcpDiscoverySpi.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Discovery/Tcp/TcpDiscoverySpi.cs new file mode 100644 index 0000000..ea946e8 --- /dev/null +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Discovery/Tcp/TcpDiscoverySpi.cs @@ -0,0 +1,144 @@ +/* + * 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.Discovery.Tcp +{ + using System; + using System.ComponentModel; + using Apache.Ignite.Core.Binary; + using Apache.Ignite.Core.Impl.Binary; + + /// + /// TCP discover service provider. + /// + public class TcpDiscoverySpi : IDiscoverySpi + { + /// + /// Default socket timeout. + /// + public static readonly TimeSpan DefaultSocketTimeout = TimeSpan.FromMilliseconds(5000); + + /// + /// Default acknowledgement timeout. + /// + public static readonly TimeSpan DefaultAckTimeout = TimeSpan.FromMilliseconds(5000); + + /// + /// Default maximum acknowledgement timeout. + /// + public static readonly TimeSpan DefaultMaxAckTimeout = TimeSpan.FromMinutes(10); + + /// + /// Default network timeout. + /// + public static readonly TimeSpan DefaultNetworkTimeout = TimeSpan.FromMilliseconds(5000); + + /// + /// Default join timeout. + /// + public static readonly TimeSpan DefaultJoinTimeout = TimeSpan.Zero; + + /// + /// Initializes a new instance of the class. + /// + public TcpDiscoverySpi() + { + SocketTimeout = DefaultSocketTimeout; + AckTimeout = DefaultAckTimeout; + MaxAckTimeout = DefaultMaxAckTimeout; + NetworkTimeout = DefaultNetworkTimeout; + JoinTimeout = DefaultJoinTimeout; + } + + /// + /// Initializes a new instance of the class. + /// + /// The reader. + internal TcpDiscoverySpi(BinaryReader reader) + { + IpFinder = reader.ReadBoolean() ? TcpDiscoveryIpFinderBase.ReadInstance(reader) : null; + + SocketTimeout = reader.ReadLongAsTimespan(); + AckTimeout = reader.ReadLongAsTimespan(); + MaxAckTimeout = reader.ReadLongAsTimespan(); + NetworkTimeout = reader.ReadLongAsTimespan(); + JoinTimeout = reader.ReadLongAsTimespan(); + } + + /// + /// Gets or sets the IP finder which defines how nodes will find each other on the network. + /// + public ITcpDiscoveryIpFinder IpFinder { get; set; } + + /// + /// Gets or sets the socket timeout. + /// + [DefaultValue(typeof(TimeSpan), "00:00:05")] + public TimeSpan SocketTimeout { get; set; } + + /// + /// Gets or sets the timeout for receiving acknowledgement for sent message. + /// + [DefaultValue(typeof(TimeSpan), "00:00:05")] + public TimeSpan AckTimeout { get; set; } + + /// + /// Gets or sets the maximum timeout for receiving acknowledgement for sent message. + /// + [DefaultValue(typeof(TimeSpan), "00:10:00")] + public TimeSpan MaxAckTimeout { get; set; } + + /// + /// Gets or sets the network timeout. + /// + [DefaultValue(typeof(TimeSpan), "00:00:05")] + public TimeSpan NetworkTimeout { get; set; } + + /// + /// Gets or sets the join timeout. + /// + public TimeSpan JoinTimeout { get; set; } + + /// + /// Writes this instance to the specified writer. + /// + internal void Write(IBinaryRawWriter writer) + { + var ipFinder = IpFinder; + + if (ipFinder != null) + { + writer.WriteBoolean(true); + + var finder = ipFinder as TcpDiscoveryIpFinderBase; + + if (finder == null) + throw new InvalidOperationException("Unsupported IP finder: " + ipFinder.GetType()); + + finder.Write(writer); + } + else + writer.WriteBoolean(false); + + writer.WriteLong((long) SocketTimeout.TotalMilliseconds); + writer.WriteLong((long) AckTimeout.TotalMilliseconds); + writer.WriteLong((long) MaxAckTimeout.TotalMilliseconds); + writer.WriteLong((long) NetworkTimeout.TotalMilliseconds); + writer.WriteLong((long) JoinTimeout.TotalMilliseconds); + } + } +} http://git-wip-us.apache.org/repos/asf/ignite/blob/ee20f1d9/modules/platforms/dotnet/Apache.Ignite.Core/IIgnite.cs ---------------------------------------------------------------------- diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/IIgnite.cs b/modules/platforms/dotnet/Apache.Ignite.Core/IIgnite.cs index a85e24c..d18e790 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/IIgnite.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/IIgnite.cs @@ -21,6 +21,7 @@ namespace Apache.Ignite.Core using System.Diagnostics.CodeAnalysis; using Apache.Ignite.Core.Binary; using Apache.Ignite.Core.Cache; + using Apache.Ignite.Core.Cache.Configuration; using Apache.Ignite.Core.Cluster; using Apache.Ignite.Core.Common; using Apache.Ignite.Core.Compute; @@ -93,6 +94,15 @@ namespace Apache.Ignite.Core ICache GetOrCreateCache(string name); /// + /// Gets existing cache with the given name or creates new one using provided configuration. + /// + /// Cache key type. + /// Cache value type. + /// Cache configuration. + /// Existing or newly created cache. + ICache GetOrCreateCache(CacheConfiguration configuration); + + /// /// Dynamically starts new cache using template configuration. /// /// Cache key type. @@ -102,8 +112,17 @@ namespace Apache.Ignite.Core ICache CreateCache(string name); /// - /// Destroys dynamically created (with or - /// ) cache. + /// Dynamically starts new cache using provided configuration. + /// + /// Cache key type. + /// Cache value type. + /// Cache configuration. + /// Existing or newly created cache. + ICache CreateCache(CacheConfiguration configuration); + + /// + /// Destroys dynamically created (with or + /// ) cache. /// /// The name of the cache to stop. void DestroyCache(string name); @@ -171,5 +190,11 @@ namespace Apache.Ignite.Core /// or null if it does not exist and create flag is not set. /// If atomic long could not be fetched or created. IAtomicLong GetAtomicLong(string name, long initialValue, bool create); + + /// + /// Gets the configuration of this Ignite instance. + /// + [SuppressMessage("Microsoft.Design", "CA1024:UsePropertiesWhereAppropriate", Justification = "Semantics.")] + IgniteConfiguration GetConfiguration(); } } http://git-wip-us.apache.org/repos/asf/ignite/blob/ee20f1d9/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 a4c37d1..1dd22ea 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfiguration.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/IgniteConfiguration.cs @@ -15,12 +15,27 @@ * limitations under the License. */ -namespace Apache.Ignite.Core +#pragma warning disable 618 // deprecated SpringConfigUrl + namespace Apache.Ignite.Core { + using System; using System.Collections.Generic; + using System.ComponentModel; + using System.Diagnostics; using System.Diagnostics.CodeAnalysis; + using System.IO; + using System.Linq; using Apache.Ignite.Core.Binary; + using Apache.Ignite.Core.Cache.Configuration; + using Apache.Ignite.Core.Discovery; + using Apache.Ignite.Core.Discovery.Tcp; + using Apache.Ignite.Core.Events; + using Apache.Ignite.Core.Impl; + using Apache.Ignite.Core.Impl.Binary; + using Apache.Ignite.Core.Impl.Common; using Apache.Ignite.Core.Lifecycle; + using BinaryReader = Apache.Ignite.Core.Impl.Binary.BinaryReader; + using BinaryWriter = Apache.Ignite.Core.Impl.Binary.BinaryWriter; /// /// Grid configuration. @@ -38,40 +53,238 @@ namespace Apache.Ignite.Core public const int DefaultJvmMaxMem = 1024; /// - /// Default constructor. + /// Default metrics expire time. + /// + public static readonly TimeSpan DefaultMetricsExpireTime = TimeSpan.MaxValue; + + /// + /// Default metrics history size. + /// + public const int DefaultMetricsHistorySize = 10000; + + /// + /// Default metrics log frequency. + /// + public static readonly TimeSpan DefaultMetricsLogFrequency = TimeSpan.FromMilliseconds(60000); + + /// + /// Default metrics update frequency. + /// + public static readonly TimeSpan DefaultMetricsUpdateFrequency = TimeSpan.FromMilliseconds(2000); + + /// + /// Default network timeout. + /// + public static readonly TimeSpan DefaultNetworkTimeout = TimeSpan.FromMilliseconds(5000); + + /// + /// Default network retry delay. + /// + public static readonly TimeSpan DefaultNetworkSendRetryDelay = TimeSpan.FromMilliseconds(1000); + + /// + /// Default network retry count. + /// + public const int DefaultNetworkSendRetryCount = 3; + + /// + /// Initializes a new instance of the class. /// public IgniteConfiguration() { JvmInitialMemoryMb = DefaultJvmInitMem; JvmMaxMemoryMb = DefaultJvmMaxMem; + + MetricsExpireTime = DefaultMetricsExpireTime; + MetricsHistorySize = DefaultMetricsHistorySize; + MetricsLogFrequency = DefaultMetricsLogFrequency; + MetricsUpdateFrequency = DefaultMetricsUpdateFrequency; + NetworkTimeout = DefaultNetworkTimeout; + NetworkSendRetryCount = DefaultNetworkSendRetryCount; + NetworkSendRetryDelay = DefaultNetworkSendRetryDelay; } /// - /// Copying constructor. + /// Initializes a new instance of the class. /// - /// Configuration. - internal IgniteConfiguration(IgniteConfiguration cfg) + /// The configuration to copy. + public IgniteConfiguration(IgniteConfiguration configuration) { - SpringConfigUrl = cfg.SpringConfigUrl; - JvmDllPath = cfg.JvmDllPath; - IgniteHome = cfg.IgniteHome; - JvmClasspath = cfg.JvmClasspath; - SuppressWarnings = cfg.SuppressWarnings; + IgniteArgumentCheck.NotNull(configuration, "configuration"); + + CopyLocalProperties(configuration); + + using (var stream = IgniteManager.Memory.Allocate().GetStream()) + { + var marsh = new Marshaller(configuration.BinaryConfiguration); + + configuration.WriteCore(marsh.StartMarshal(stream)); + + stream.SynchronizeOutput(); + + stream.Seek(0, SeekOrigin.Begin); + + ReadCore(marsh.StartUnmarshal(stream)); + } + } + + /// + /// Initializes a new instance of the class from a reader. + /// + /// The binary reader. + internal IgniteConfiguration(BinaryReader binaryReader) + { + Read(binaryReader); + } + + /// + /// Writes this instance to a writer. + /// + /// The writer. + internal void Write(BinaryWriter writer) + { + Debug.Assert(writer != null); + + if (!string.IsNullOrEmpty(SpringConfigUrl)) + { + // Do not write details when there is Spring config. + writer.WriteBoolean(false); + return; + } + + writer.WriteBoolean(true); // details are present + + WriteCore(writer); + } + + /// + /// Writes this instance to a writer. + /// + /// The writer. + private void WriteCore(BinaryWriter writer) + { + // Simple properties + writer.WriteBoolean(ClientMode); + writer.WriteIntArray(IncludedEventTypes == null ? null : IncludedEventTypes.ToArray()); + + writer.WriteLong((long) MetricsExpireTime.TotalMilliseconds); + writer.WriteInt(MetricsHistorySize); + writer.WriteLong((long) MetricsLogFrequency.TotalMilliseconds); + var metricsUpdateFreq = (long) MetricsUpdateFrequency.TotalMilliseconds; + writer.WriteLong(metricsUpdateFreq >= 0 ? metricsUpdateFreq : -1); + writer.WriteInt(NetworkSendRetryCount); + writer.WriteLong((long) NetworkSendRetryDelay.TotalMilliseconds); + writer.WriteLong((long) NetworkTimeout.TotalMilliseconds); + writer.WriteString(WorkDirectory); + writer.WriteString(Localhost); + + // Cache config + var caches = CacheConfiguration; + + if (caches == null) + writer.WriteInt(0); + else + { + writer.WriteInt(caches.Count); + + foreach (var cache in caches) + cache.Write(writer); + } + + // Discovery config + var disco = DiscoverySpi; + + if (disco != null) + { + writer.WriteBoolean(true); + + var tcpDisco = disco as TcpDiscoverySpi; + + if (tcpDisco == null) + throw new InvalidOperationException("Unsupported discovery SPI: " + disco.GetType()); + + tcpDisco.Write(writer); + } + else + writer.WriteBoolean(false); + } + + /// + /// Reads data from specified reader into current instance. + /// + /// The binary reader. + private void ReadCore(BinaryReader r) + { + // Simple properties + ClientMode = r.ReadBoolean(); + IncludedEventTypes = r.ReadIntArray(); - JvmOptions = cfg.JvmOptions != null ? new List(cfg.JvmOptions) : null; - Assemblies = cfg.Assemblies != null ? new List(cfg.Assemblies) : null; + MetricsExpireTime = r.ReadLongAsTimespan(); + MetricsHistorySize = r.ReadInt(); + MetricsLogFrequency = r.ReadLongAsTimespan(); + MetricsUpdateFrequency = r.ReadLongAsTimespan(); + NetworkSendRetryCount = r.ReadInt(); + NetworkSendRetryDelay = r.ReadLongAsTimespan(); + NetworkTimeout = r.ReadLongAsTimespan(); + WorkDirectory = r.ReadString(); + Localhost = r.ReadString(); - BinaryConfiguration = cfg.BinaryConfiguration != null - ? new BinaryConfiguration(cfg.BinaryConfiguration) - : null; + // Cache config + var cacheCfgCount = r.ReadInt(); + CacheConfiguration = new List(cacheCfgCount); + for (int i = 0; i < cacheCfgCount; i++) + CacheConfiguration.Add(new CacheConfiguration(r)); - LifecycleBeans = cfg.LifecycleBeans != null ? new List(cfg.LifecycleBeans) : null; + // Discovery config + DiscoverySpi = r.ReadBoolean() ? new TcpDiscoverySpi(r) : null; + } + + /// + /// Reads data from specified reader into current instance. + /// + /// The binary reader. + private void Read(BinaryReader binaryReader) + { + var r = binaryReader; + + CopyLocalProperties(r.Marshaller.Ignite.Configuration); + + ReadCore(r); + + // Misc + IgniteHome = r.ReadString(); - JvmInitialMemoryMb = cfg.JvmInitialMemoryMb; - JvmMaxMemoryMb = cfg.JvmMaxMemoryMb; + JvmInitialMemoryMb = (int) (r.ReadLong()/1024/2014); + JvmMaxMemoryMb = (int) (r.ReadLong()/1024/2014); + + // Local data (not from reader) + JvmDllPath = Process.GetCurrentProcess().Modules.OfType() + .Single(x => string.Equals(x.ModuleName, IgniteUtils.FileJvmDll, StringComparison.OrdinalIgnoreCase)) + .FileName; + } + + /// + /// Copies the local properties (properties that are not written in Write method). + /// + private void CopyLocalProperties(IgniteConfiguration cfg) + { + GridName = cfg.GridName; + BinaryConfiguration = cfg.BinaryConfiguration == null + ? null + : new BinaryConfiguration(cfg.BinaryConfiguration); + JvmClasspath = cfg.JvmClasspath; + JvmOptions = cfg.JvmOptions; + Assemblies = cfg.Assemblies; + SuppressWarnings = cfg.SuppressWarnings; + LifecycleBeans = cfg.LifecycleBeans; } /// + /// Grid name which is used if not provided in configuration file. + /// + public string GridName { get; set; } + + /// /// Gets or sets the binary configuration. /// /// @@ -80,9 +293,22 @@ namespace Apache.Ignite.Core public BinaryConfiguration BinaryConfiguration { get; set; } /// + /// Gets or sets the cache configuration. + /// + /// + /// The cache configuration. + /// + [SuppressMessage("Microsoft.Usage", "CA2227:CollectionPropertiesShouldBeReadOnly")] + public ICollection CacheConfiguration { get; set; } + + /// /// URL to Spring configuration file. /// [SuppressMessage("Microsoft.Design", "CA1056:UriPropertiesShouldNotBeStrings")] + [Obsolete("Ignite.NET can be configured natively without Spring. " + + "Setting this property will ignore all other properties except " + + "IgniteHome, Assemblies, SuppressWarnings, LifecycleBeans, JvmOptions, JvmdllPath, IgniteHome, " + + "JvmInitialMemoryMb, JvmMaxMemoryMb.")] public string SpringConfigUrl { get; set; } /// @@ -115,7 +341,7 @@ namespace Apache.Ignite.Core /// assemblies reside. /// [SuppressMessage("Microsoft.Usage", "CA2227:CollectionPropertiesShouldBeReadOnly")] - public IList Assemblies { get; set; } + public ICollection Assemblies { get; set; } /// /// Whether to suppress warnings. @@ -132,12 +358,95 @@ namespace Apache.Ignite.Core /// Initial amount of memory in megabytes given to JVM. Maps to -Xms Java option. /// Defaults to . /// + [DefaultValue(DefaultJvmInitMem)] public int JvmInitialMemoryMb { get; set; } /// /// Maximum amount of memory in megabytes given to JVM. Maps to -Xmx Java option. /// Defaults to . /// + [DefaultValue(DefaultJvmMaxMem)] public int JvmMaxMemoryMb { get; set; } + + /// + /// Gets or sets the discovery service provider. + /// Null for default discovery. + /// + public IDiscoverySpi DiscoverySpi { get; set; } + + /// + /// 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; } + + /// + /// Gets or sets a set of event types () to be recorded by Ignite. + /// + [SuppressMessage("Microsoft.Usage", "CA2227:CollectionPropertiesShouldBeReadOnly")] + public ICollection IncludedEventTypes { get; set; } + + /// + /// Gets or sets the time after which a certain metric value is considered expired. + /// + [DefaultValue(typeof(TimeSpan), "10675199.02:48:05.4775807")] + public TimeSpan MetricsExpireTime { get; set; } + + /// + /// Gets or sets the number of metrics kept in history to compute totals and averages. + /// + [DefaultValue(DefaultMetricsHistorySize)] + public int MetricsHistorySize { get; set; } + + /// + /// Gets or sets the frequency of metrics log print out. + /// to disable metrics print out. + /// + [DefaultValue(typeof(TimeSpan), "00:01:00")] + public TimeSpan MetricsLogFrequency { get; set; } + + /// + /// Gets or sets the job metrics update frequency. + /// to update metrics on job start/finish. + /// Negative value to never update metrics. + /// + [DefaultValue(typeof(TimeSpan), "00:00:02")] + public TimeSpan MetricsUpdateFrequency { get; set; } + + /// + /// Gets or sets the network send retry count. + /// + [DefaultValue(DefaultNetworkSendRetryCount)] + public int NetworkSendRetryCount { get; set; } + + /// + /// Gets or sets the network send retry delay. + /// + [DefaultValue(typeof(TimeSpan), "00:00:01")] + public TimeSpan NetworkSendRetryDelay { get; set; } + + /// + /// Gets or sets the network timeout. + /// + [DefaultValue(typeof(TimeSpan), "00:00:05")] + public TimeSpan NetworkTimeout { get; set; } + + /// + /// Gets or sets the work directory. + /// If not provided, a folder under will be used. + /// + public string WorkDirectory { get; set; } + + /// + /// Gets or sets system-wide local address or host for all Ignite components to bind to. + /// If provided it will override all default local bind settings within Ignite. + /// + /// If null then Ignite tries to use local wildcard address.That means that all services + /// will be available on all network interfaces of the host machine. + /// + /// It is strongly recommended to set this parameter for all production environments. + /// + public string Localhost { get; set; } } } http://git-wip-us.apache.org/repos/asf/ignite/blob/ee20f1d9/modules/platforms/dotnet/Apache.Ignite.Core/Ignition.cs ---------------------------------------------------------------------- diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Ignition.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Ignition.cs index 3a27ad1..0549010 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Ignition.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Ignition.cs @@ -15,6 +15,7 @@ * limitations under the License. */ +#pragma warning disable 618 // deprecated SpringConfigUrl namespace Apache.Ignite.Core { using System; @@ -41,7 +42,7 @@ namespace Apache.Ignite.Core /// /// This class defines a factory for the main Ignite API. ///

- /// Use method to start Ignite with default configuration. + /// Use method to start Ignite with default configuration. /// /// All members are thread-safe and may be used concurrently from multiple threads. ///

@@ -51,9 +52,6 @@ namespace Apache.Ignite.Core internal const string EnvIgniteSpringConfigUrlPrefix = "IGNITE_SPRING_CONFIG_URL_PREFIX"; /** */ - private const string DefaultCfg = "config/default-config.xml"; - - /** */ private static readonly object SyncRoot = new object(); /** GC warning flag. */ @@ -125,15 +123,6 @@ namespace Apache.Ignite.Core { IgniteArgumentCheck.NotNull(cfg, "cfg"); - // Copy configuration to avoid changes to user-provided instance. - IgniteConfigurationEx cfgEx = cfg as IgniteConfigurationEx; - - cfg = cfgEx == null ? new IgniteConfiguration(cfg) : new IgniteConfigurationEx(cfgEx); - - // Set default Spring config if needed. - if (cfg.SpringConfigUrl == null) - cfg.SpringConfigUrl = DefaultCfg; - lock (SyncRoot) { // 1. Check GC settings. @@ -146,9 +135,11 @@ namespace Apache.Ignite.Core IgniteManager.CreateJvmContext(cfg, cbs); - var gridName = cfgEx != null ? cfgEx.GridName : null; + var gridName = cfg.GridName; - var cfgPath = Environment.GetEnvironmentVariable(EnvIgniteSpringConfigUrlPrefix) + cfg.SpringConfigUrl; + var cfgPath = cfg.SpringConfigUrl == null + ? null + : Environment.GetEnvironmentVariable(EnvIgniteSpringConfigUrlPrefix) + cfg.SpringConfigUrl; // 3. Create startup object which will guide us through the rest of the process. _startup = new Startup(cfg, cbs); @@ -229,7 +220,7 @@ namespace Apache.Ignite.Core { BinaryReader reader = BinaryUtils.Marshaller.StartUnmarshal(inStream); - PrepareConfiguration(reader); + PrepareConfiguration(reader, outStream); PrepareLifecycleBeans(reader, outStream, handleRegistry); } @@ -245,7 +236,8 @@ namespace Apache.Ignite.Core /// Preapare configuration. ///
/// Reader. - private static void PrepareConfiguration(BinaryReader reader) + /// Response stream. + private static void PrepareConfiguration(BinaryReader reader, PlatformMemoryStream outStream) { // 1. Load assemblies. IgniteConfiguration cfg = _startup.Configuration; @@ -264,6 +256,9 @@ namespace Apache.Ignite.Core cfg.BinaryConfiguration = binaryCfg; _startup.Marshaller = new Marshaller(cfg.BinaryConfiguration); + + // 3. Send configuration details to Java + cfg.Write(_startup.Marshaller.StartMarshal(outStream)); } /// http://git-wip-us.apache.org/repos/asf/ignite/blob/ee20f1d9/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/Binary.cs ---------------------------------------------------------------------- diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/Binary.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/Binary.cs index 9b43564..efe1df4 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/Binary.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/Binary.cs @@ -104,9 +104,6 @@ namespace Apache.Ignite.Core.Impl.Binary if (obj0 == null) throw new ArgumentException("Unsupported object type: " + obj.GetType()); - if (obj0 is BinaryEnum) - throw new InvalidOperationException("Builder cannot be created for enum."); - IBinaryTypeDescriptor desc = _marsh.GetDescriptor(true, obj0.TypeId); return Builder0(null, obj0, desc); http://git-wip-us.apache.org/repos/asf/ignite/blob/ee20f1d9/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryReaderExtensions.cs ---------------------------------------------------------------------- diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryReaderExtensions.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryReaderExtensions.cs index c3dcc3a..f3f8457 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryReaderExtensions.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/BinaryReaderExtensions.cs @@ -17,6 +17,7 @@ namespace Apache.Ignite.Core.Impl.Binary { + using System; using System.Collections.Generic; using Apache.Ignite.Core.Binary; @@ -48,5 +49,23 @@ namespace Apache.Ignite.Core.Impl.Binary { return (Dictionary) reader.ReadDictionary(size => new Dictionary(size)); } + + /// + /// Reads long as timespan with range checks. + /// + /// The reader. + /// TimeSpan. + public static TimeSpan ReadLongAsTimespan(this IBinaryRawReader reader) + { + long ms = reader.ReadLong(); + + if (ms >= TimeSpan.MaxValue.TotalMilliseconds) + return TimeSpan.MaxValue; + + if (ms <= TimeSpan.MinValue.TotalMilliseconds) + return TimeSpan.MinValue; + + return TimeSpan.FromMilliseconds(ms); + } } } http://git-wip-us.apache.org/repos/asf/ignite/blob/ee20f1d9/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/JavaTypes.cs ---------------------------------------------------------------------- diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/JavaTypes.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/JavaTypes.cs new file mode 100644 index 0000000..fccbfae --- /dev/null +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/JavaTypes.cs @@ -0,0 +1,92 @@ +/* + * 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.Binary +{ + using System; + using System.Collections.Generic; + using System.Linq; + + /// + /// Provides mapping between Java and .NET basic types. + /// + internal static class JavaTypes + { + /** */ + private static readonly Dictionary NetToJava = new Dictionary + { + {typeof (bool), "java.lang.Boolean"}, + {typeof (byte), "java.lang.Byte"}, + {typeof (sbyte), "java.lang.Byte"}, + {typeof (short), "java.lang.Short"}, + {typeof (ushort), "java.lang.Short"}, + {typeof (char), "java.lang.Character"}, + {typeof (int), "java.lang.Integer"}, + {typeof (uint), "java.lang.Integer"}, + {typeof (long), "java.lang.Long"}, + {typeof (ulong), "java.lang.Long"}, + {typeof (float), "java.lang.Float"}, + {typeof (double), "java.lang.Double"}, + {typeof (string), "java.lang.String"}, + {typeof (decimal), "java.math.BigDecimal"}, + {typeof (Guid), "java.util.UUID"} + }; + + /** */ + private static readonly Dictionary JavaToNet = + NetToJava.GroupBy(x => x.Value).ToDictionary(g => g.Key, g => g.First().Key); + + /** */ + private static readonly string MappedTypes = string.Join(", ", NetToJava.Keys.Select(x => x.Name)); + + /// + /// Gets the corresponding Java type name. + /// + public static string GetJavaTypeName(Type type) + { + if (type == null) + return null; + + string res; + + return NetToJava.TryGetValue(type, out res) ? res : null; + } + + /// + /// Gets .NET type that corresponds to specified Java type name. + /// + /// Name of the java type. + /// + public static Type GetDotNetType(string javaTypeName) + { + if (string.IsNullOrEmpty(javaTypeName)) + return null; + + Type res; + + return JavaToNet.TryGetValue(javaTypeName, out res) ? res : null; + } + + /// + /// Gets the supported types as a comma-separated string. + /// + public static string SupportedTypesString + { + get { return MappedTypes; } + } + } +} http://git-wip-us.apache.org/repos/asf/ignite/blob/ee20f1d9/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/Marshaller.cs ---------------------------------------------------------------------- diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/Marshaller.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/Marshaller.cs index 4274744..73f3618 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/Marshaller.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Binary/Marshaller.cs @@ -88,11 +88,11 @@ namespace Apache.Ignite.Core.Impl.Binary foreach (BinaryTypeConfiguration typeCfg in typeCfgs) AddUserType(cfg, typeCfg, typeResolver, dfltSerializer); - ICollection types = cfg.Types; + var typeNames = cfg.Types; - if (types != null) - foreach (string type in types) - AddUserType(cfg, new BinaryTypeConfiguration(type), typeResolver, dfltSerializer); + if (typeNames != null) + foreach (string typeName in typeNames) + AddUserType(cfg, new BinaryTypeConfiguration(typeName), typeResolver, dfltSerializer); if (cfg.DefaultSerializer == null) cfg.DefaultSerializer = dfltSerializer; http://git-wip-us.apache.org/repos/asf/ignite/blob/ee20f1d9/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/CacheImpl.cs ---------------------------------------------------------------------- diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/CacheImpl.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/CacheImpl.cs index f55863e..d1296ec 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/CacheImpl.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/CacheImpl.cs @@ -25,6 +25,7 @@ namespace Apache.Ignite.Core.Impl.Cache using System.Threading.Tasks; using Apache.Ignite.Core.Binary; using Apache.Ignite.Core.Cache; + using Apache.Ignite.Core.Cache.Configuration; using Apache.Ignite.Core.Cache.Expiry; using Apache.Ignite.Core.Cache.Query; using Apache.Ignite.Core.Cache.Query.Continuous; @@ -68,7 +69,7 @@ namespace Apache.Ignite.Core.Impl.Cache /** Async instance. */ private readonly Lazy> _asyncInstance; - + /// /// Constructor. /// @@ -152,6 +153,12 @@ namespace Apache.Ignite.Core.Impl.Cache } /** */ + public CacheConfiguration GetConfiguration() + { + return DoInOp((int) CacheOp.GetConfig, stream => new CacheConfiguration(Marshaller.StartUnmarshal(stream))); + } + + /** */ public bool IsEmpty() { return GetSize() == 0; http://git-wip-us.apache.org/repos/asf/ignite/blob/ee20f1d9/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/CacheOp.cs ---------------------------------------------------------------------- diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/CacheOp.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/CacheOp.cs index 1709dc5..61ccb5f 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/CacheOp.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/CacheOp.cs @@ -59,6 +59,7 @@ namespace Apache.Ignite.Core.Impl.Cache RemoveBool = 35, RemoveObj = 36, Replace2 = 37, - Replace3 = 38 + Replace3 = 38, + GetConfig = 39 } } \ No newline at end of file http://git-wip-us.apache.org/repos/asf/ignite/blob/ee20f1d9/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/Store/CacheStore.cs ---------------------------------------------------------------------- diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/Store/CacheStore.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/Store/CacheStore.cs index 5aa806b..7785280 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/Store/CacheStore.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Cache/Store/CacheStore.cs @@ -101,15 +101,25 @@ namespace Apache.Ignite.Core.Impl.Cache.Store { using (var stream = IgniteManager.Memory.Get(memPtr).GetStream()) { - var reader = BinaryUtils.Marshaller.StartUnmarshal(stream, BinaryMode.KeepBinary); + var reader = BinaryUtils.Marshaller.StartUnmarshal(stream); - var className = reader.ReadString(); var convertBinary = reader.ReadBoolean(); - var propertyMap = reader.ReadDictionaryAsGeneric(); + var factory = reader.ReadObject>(); - var store = IgniteUtils.CreateInstance(className); + ICacheStore store; + + if (factory != null) + store = factory.CreateInstance(); + else + { + var className = reader.ReadString(); + var propertyMap = reader.ReadDictionaryAsGeneric(); + + store = IgniteUtils.CreateInstance(className); + + IgniteUtils.SetProperties(store, propertyMap); + } - IgniteUtils.SetProperties(store, propertyMap); return new CacheStore(store, convertBinary, registry); } http://git-wip-us.apache.org/repos/asf/ignite/blob/ee20f1d9/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Ignite.cs ---------------------------------------------------------------------- diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Ignite.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Ignite.cs index ffc8be8..9d27117 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Ignite.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Ignite.cs @@ -25,6 +25,7 @@ namespace Apache.Ignite.Core.Impl using System.Linq; using Apache.Ignite.Core.Binary; using Apache.Ignite.Core.Cache; + using Apache.Ignite.Core.Cache.Configuration; using Apache.Ignite.Core.Cluster; using Apache.Ignite.Core.Compute; using Apache.Ignite.Core.Datastream; @@ -339,12 +340,46 @@ namespace Apache.Ignite.Core.Impl } /** */ + public ICache GetOrCreateCache(CacheConfiguration configuration) + { + IgniteArgumentCheck.NotNull(configuration, "configuration"); + + using (var stream = IgniteManager.Memory.Allocate().GetStream()) + { + var writer = Marshaller.StartMarshal(stream); + + configuration.Write(writer); + + stream.SynchronizeOutput(); + + return Cache(UU.ProcessorGetOrCreateCache(_proc, stream.MemoryPointer)); + } + } + + /** */ public ICache CreateCache(string name) { return Cache(UU.ProcessorCreateCache(_proc, name)); } /** */ + public ICache CreateCache(CacheConfiguration configuration) + { + IgniteArgumentCheck.NotNull(configuration, "configuration"); + + using (var stream = IgniteManager.Memory.Allocate().GetStream()) + { + var writer = Marshaller.StartMarshal(stream); + + configuration.Write(writer); + + stream.SynchronizeOutput(); + + return Cache(UU.ProcessorCreateCache(_proc, stream.MemoryPointer)); + } + } + + /** */ public void DestroyCache(string name) { UU.ProcessorDestroyCache(_proc, name); @@ -450,6 +485,19 @@ namespace Apache.Ignite.Core.Impl return new AtomicLong(nativeLong, Marshaller, name); } + /** */ + public IgniteConfiguration GetConfiguration() + { + using (var stream = IgniteManager.Memory.Allocate(1024).GetStream()) + { + UU.ProcessorGetIgniteConfiguration(_proc, stream.MemoryPointer); + + stream.SynchronizeInput(); + + return new IgniteConfiguration(_marsh.StartUnmarshal(stream)); + } + } + /// /// Gets internal projection. /// http://git-wip-us.apache.org/repos/asf/ignite/blob/ee20f1d9/modules/platforms/dotnet/Apache.Ignite.Core/Impl/IgniteConfigurationEx.cs ---------------------------------------------------------------------- diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/IgniteConfigurationEx.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/IgniteConfigurationEx.cs deleted file mode 100644 index 358e805..0000000 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/IgniteConfigurationEx.cs +++ /dev/null @@ -1,57 +0,0 @@ -/* - * 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 -{ - /// - /// Internal extensions for IgniteConfiguration. - /// - internal class IgniteConfigurationEx : IgniteConfiguration - { - /// - /// Default constructor. - /// - public IgniteConfigurationEx() - { - // No-op. - } - - /// - /// Copying constructor. - /// - /// Configuration. - public IgniteConfigurationEx(IgniteConfiguration cfg) : base(cfg) - { - // No-op. - } - - /// - /// Copying constructor. - /// - /// Configuration. - public IgniteConfigurationEx(IgniteConfigurationEx cfg) - : this((IgniteConfiguration) cfg) - { - GridName = cfg.GridName; - } - - /// - /// Grid name which is used if not provided in configuration file. - /// - public string GridName { get; set; } - } -} http://git-wip-us.apache.org/repos/asf/ignite/blob/ee20f1d9/modules/platforms/dotnet/Apache.Ignite.Core/Impl/IgniteManager.cs ---------------------------------------------------------------------- diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/IgniteManager.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/IgniteManager.cs index 6803772..a61edf4 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/IgniteManager.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/IgniteManager.cs @@ -50,7 +50,7 @@ namespace Apache.Ignite.Core.Impl private static JvmConfiguration _jvmCfg; /** Memory manager. */ - private static PlatformMemoryManager _mem; + private static readonly PlatformMemoryManager Mem = new PlatformMemoryManager(1024); /// /// Create JVM. @@ -86,7 +86,6 @@ namespace Apache.Ignite.Core.Impl { _ctx = ctx; _jvmCfg = jvmCfg; - _mem = new PlatformMemoryManager(1024); } } } @@ -96,7 +95,7 @@ namespace Apache.Ignite.Core.Impl /// internal static PlatformMemoryManager Memory { - get { return _mem; } + get { return Mem; } } /// http://git-wip-us.apache.org/repos/asf/ignite/blob/ee20f1d9/modules/platforms/dotnet/Apache.Ignite.Core/Impl/IgniteProxy.cs ---------------------------------------------------------------------- diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/IgniteProxy.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/IgniteProxy.cs index 16062e2..46bc3ca 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/IgniteProxy.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/IgniteProxy.cs @@ -22,6 +22,7 @@ namespace Apache.Ignite.Core.Impl using System.Diagnostics.CodeAnalysis; using Apache.Ignite.Core.Binary; using Apache.Ignite.Core.Cache; + using Apache.Ignite.Core.Cache.Configuration; using Apache.Ignite.Core.Cluster; using Apache.Ignite.Core.Compute; using Apache.Ignite.Core.Datastream; @@ -232,19 +233,28 @@ namespace Apache.Ignite.Core.Impl } /** */ + public ICache GetOrCreateCache(CacheConfiguration configuration) + { + return _ignite.GetOrCreateCache(configuration); + } + + /** */ public ICache CreateCache(string name) { return _ignite.CreateCache(name); } /** */ + public ICache CreateCache(CacheConfiguration configuration) + { + return _ignite.CreateCache(configuration); + } public void DestroyCache(string name) { _ignite.DestroyCache(name); } /** */ - public IClusterNode GetLocalNode() { return _ignite.GetCluster().GetLocalNode(); @@ -324,6 +334,12 @@ namespace Apache.Ignite.Core.Impl } /** */ + public IgniteConfiguration GetConfiguration() + { + return _ignite.GetConfiguration(); + } + + /** */ public void WriteBinary(IBinaryWriter writer) { // No-op. http://git-wip-us.apache.org/repos/asf/ignite/blob/ee20f1d9/modules/platforms/dotnet/Apache.Ignite.Core/Impl/IgniteUtils.cs ---------------------------------------------------------------------- diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/IgniteUtils.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/IgniteUtils.cs index fe4548c..3206fc8 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/IgniteUtils.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/IgniteUtils.cs @@ -47,7 +47,7 @@ namespace Apache.Ignite.Core.Impl private static readonly string[] JvmDllLookupPaths = {@"jre\bin\server", @"jre\bin\default"}; /** File: jvm.dll. */ - private const string FileJvmDll = "jvm.dll"; + internal const string FileJvmDll = "jvm.dll"; /** File: Ignite.Common.dll. */ internal const string FileIgniteJniDll = "ignite.common.dll"; http://git-wip-us.apache.org/repos/asf/ignite/blob/ee20f1d9/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Memory/PlatformRawMemory.cs ---------------------------------------------------------------------- diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Memory/PlatformRawMemory.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Memory/PlatformRawMemory.cs index 851d24f..8e54261 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Memory/PlatformRawMemory.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Memory/PlatformRawMemory.cs @@ -85,7 +85,7 @@ namespace Apache.Ignite.Core.Impl.Memory /** */ public void Release() { - throw new NotSupportedException(); + // Memory can only be released by native platform. } } } \ No newline at end of file http://git-wip-us.apache.org/repos/asf/ignite/blob/ee20f1d9/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Transactions/TransactionsImpl.cs ---------------------------------------------------------------------- diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Transactions/TransactionsImpl.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Transactions/TransactionsImpl.cs index 229ff8c..51a49d0 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Transactions/TransactionsImpl.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Transactions/TransactionsImpl.cs @@ -70,7 +70,7 @@ namespace Apache.Ignite.Core.Impl.Transactions concurrency = (TransactionConcurrency) reader.ReadInt(); isolation = (TransactionIsolation) reader.ReadInt(); - timeout = TimeSpan.FromMilliseconds(reader.ReadLong()); + timeout = reader.ReadLongAsTimespan(); }); _dfltConcurrency = concurrency; http://git-wip-us.apache.org/repos/asf/ignite/blob/ee20f1d9/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/IgniteJniNativeMethods.cs ---------------------------------------------------------------------- diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/IgniteJniNativeMethods.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/IgniteJniNativeMethods.cs index 7a73bee..17df94a 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/IgniteJniNativeMethods.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/IgniteJniNativeMethods.cs @@ -24,7 +24,7 @@ namespace Apache.Ignite.Core.Impl.Unmanaged /// Ignite JNI native methods. /// [SuppressUnmanagedCodeSecurity] - internal unsafe static class IgniteJniNativeMethods + internal static unsafe class IgniteJniNativeMethods { [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteReallocate")] public static extern int Reallocate(long memPtr, int cap); @@ -52,9 +52,15 @@ namespace Apache.Ignite.Core.Impl.Unmanaged [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteProcessorCreateCache")] public static extern void* ProcessorCreateCache(void* ctx, void* obj, sbyte* name); + [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteProcessorCreateCacheFromConfig")] + public static extern void* ProcessorCreateCacheFromConfig(void* ctx, void* obj, long memPtr); + [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteProcessorGetOrCreateCache")] public static extern void* ProcessorGetOrCreateCache(void* ctx, void* obj, sbyte* name); + [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteProcessorGetOrCreateCacheFromConfig")] + public static extern void* ProcessorGetOrCreateCacheFromConfig(void* ctx, void* obj, long memPtr); + [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteProcessorDestroyCache")] public static extern void ProcessorDestroyCache(void* ctx, void* obj, sbyte* name); @@ -87,6 +93,9 @@ namespace Apache.Ignite.Core.Impl.Unmanaged public static extern void* ProcessorAtomicLong(void* ctx, void* obj, sbyte* name, long initVal, [MarshalAs(UnmanagedType.U1)] bool create); + [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteProcessorGetIgniteConfiguration")] + public static extern void ProcessorGetIgniteConfiguration(void* ctx, void* obj, long memPtr); + [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteTargetInStreamOutLong")] public static extern long TargetInStreamOutLong(void* ctx, void* target, int opType, long memPtr); http://git-wip-us.apache.org/repos/asf/ignite/blob/ee20f1d9/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedUtils.cs ---------------------------------------------------------------------- diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedUtils.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedUtils.cs index f460c9f..ad62f38 100644 --- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedUtils.cs +++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedUtils.cs @@ -21,7 +21,6 @@ namespace Apache.Ignite.Core.Impl.Unmanaged using System.Diagnostics.CodeAnalysis; using System.Runtime.InteropServices; using Apache.Ignite.Core.Common; - using JNI = IgniteJniNativeMethods; /// @@ -102,7 +101,7 @@ namespace Apache.Ignite.Core.Impl.Unmanaged { JNI.IgnitionStopAll(ctx, cancel); } - + internal static void ProcessorReleaseStart(IUnmanagedTarget target) { JNI.ProcessorReleaseStart(target.Context, target.Target); @@ -147,6 +146,13 @@ namespace Apache.Ignite.Core.Impl.Unmanaged } } + internal static IUnmanagedTarget ProcessorCreateCache(IUnmanagedTarget target, long memPtr) + { + void* res = JNI.ProcessorCreateCacheFromConfig(target.Context, target.Target, memPtr); + + return target.ChangeTarget(res); + } + internal static IUnmanagedTarget ProcessorGetOrCreateCache(IUnmanagedTarget target, string name) { sbyte* name0 = IgniteUtils.StringToUtf8Unmanaged(name); @@ -163,6 +169,13 @@ namespace Apache.Ignite.Core.Impl.Unmanaged } } + internal static IUnmanagedTarget ProcessorGetOrCreateCache(IUnmanagedTarget target, long memPtr) + { + void* res = JNI.ProcessorGetOrCreateCacheFromConfig(target.Context, target.Target, memPtr); + + return target.ChangeTarget(res); + } + internal static void ProcessorDestroyCache(IUnmanagedTarget target, string name) { sbyte* name0 = IgniteUtils.StringToUtf8Unmanaged(name); @@ -268,6 +281,11 @@ namespace Apache.Ignite.Core.Impl.Unmanaged } } + internal static void ProcessorGetIgniteConfiguration(IUnmanagedTarget target, long memPtr) + { + JNI.ProcessorGetIgniteConfiguration(target.Context, target.Target, memPtr); + } + #endregion #region NATIVE METHODS: TARGET http://git-wip-us.apache.org/repos/asf/ignite/blob/ee20f1d9/parent/pom.xml ---------------------------------------------------------------------- diff --git a/parent/pom.xml b/parent/pom.xml index c0f49c8..53eefdd 100644 --- a/parent/pom.xml +++ b/parent/pom.xml @@ -837,6 +837,7 @@ **/*.slnrel **/*.DotSettings **/*.FxCop + **/*.ruleset **/*.csproj **/*.csprojrel **/*.vcxproj